Explorar el Código

Merge trunk into HA branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1207490 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers hace 13 años
padre
commit
73b3de6204
Se han modificado 100 ficheros con 21693 adiciones y 436 borrados
  1. 17 1
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 5 1
      hadoop-common-project/hadoop-common/src/main/docs/changes/changes2html.pl
  3. 2 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  4. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
  5. 12 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  6. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  7. 5 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
  8. 107 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java
  9. 16 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/CachedDNSToSwitchMapping.java
  10. 6 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNSToSwitchMapping.java
  11. 98 57
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java
  12. 83 17
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/StaticMapping.java
  13. 41 12
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java
  14. 103 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java
  15. 53 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java
  16. 13 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  17. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/docs/changes/changes2html.pl
  18. 27 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/docs/src/documentation/content/xdocs/site.xml
  19. 1555 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/docs/src/documentation/content/xdocs/webhdfs.xml
  20. 17153 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/DatanodeProtocolProtos.java
  21. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/browseBlock.jsp
  22. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/browseDirectory.jsp
  23. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/tail.jsp
  24. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp
  25. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp
  26. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp
  27. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/nn_browsedfscontent.jsp
  28. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.jsp
  29. 344 0
      hadoop-hdfs-project/hadoop-hdfs/src/proto/DatanodeProtocol.proto
  30. 27 0
      hadoop-mapreduce-project/CHANGES.txt
  31. 0 4
      hadoop-mapreduce-project/build.xml
  32. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppView.java
  33. 4 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
  34. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
  35. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java
  36. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
  37. 6 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
  38. 6 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
  39. 9 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java
  40. 2 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java
  41. 8 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
  42. 18 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
  43. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  44. 2 2
      hadoop-mapreduce-project/hadoop-yarn/bin/yarn-daemon.sh
  45. 2 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocol.java
  46. 3 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java
  47. 5 4
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManager.java
  48. 5 6
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
  49. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
  50. 2 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsRequest.java
  51. 2 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsRequest.java
  52. 2 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesRequest.java
  53. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesResponse.java
  54. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusResponse.java
  55. 2 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationRequest.java
  56. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationResponse.java
  57. 2 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoRequest.java
  58. 42 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerResponse.java
  59. 2 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerResponse.java
  60. 2 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationResponse.java
  61. 2 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMResponse.java
  62. 2 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
  63. 29 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java
  64. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
  65. 1 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
  66. 12 7
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
  67. 0 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeHealthStatus.java
  68. 13 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java
  69. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java
  70. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  71. 5 5
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java
  72. 2 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java
  73. 2 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java
  74. 2 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ApplicationTokenSelector.java
  75. 2 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientTokenSelector.java
  76. 4 4
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
  77. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
  78. 2 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.java
  79. 29 32
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java
  80. 34 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
  81. 4 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
  82. 12 6
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  83. 9 3
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
  84. 4 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.java
  85. 46 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java
  86. 9 9
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/Federation.apt.vm
  87. 1626 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebHDFS.apt.vm
  88. 4 0
      hadoop-mapreduce-project/src/contrib/gridmix/ivy.xml
  89. 0 52
      hadoop-mapreduce-project/src/contrib/streaming/build.xml
  90. 0 98
      hadoop-mapreduce-project/src/contrib/streaming/ivy.xml
  91. 0 17
      hadoop-mapreduce-project/src/contrib/streaming/ivy/libraries.properties
  92. 5 1
      hadoop-mapreduce-project/src/docs/changes/changes2html.pl
  93. 1 3
      hadoop-mapreduce-project/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
  94. 1 0
      hadoop-mapreduce-project/src/webapps/job/analysejobhistory.jsp
  95. 1 0
      hadoop-mapreduce-project/src/webapps/job/job_authorization_error.jsp
  96. 1 0
      hadoop-mapreduce-project/src/webapps/job/jobblacklistedtrackers.jsp
  97. 1 1
      hadoop-mapreduce-project/src/webapps/job/jobconf.jsp
  98. 1 0
      hadoop-mapreduce-project/src/webapps/job/jobconf_history.jsp
  99. 1 0
      hadoop-mapreduce-project/src/webapps/job/jobdetails.jsp
  100. 1 0
      hadoop-mapreduce-project/src/webapps/job/jobdetailshistory.jsp

+ 17 - 1
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -3,7 +3,12 @@ Hadoop Change Log
 Trunk (unreleased changes)
 
   INCOMPATIBLE CHANGES
-   
+
+  NEW FEATURES
+
+    HADOOP-7777 Implement a base class for DNSToSwitchMapping implementations 
+    that can offer extra topology information. (stevel)
+
   IMPROVEMENTS
 
     HADOOP-7595. Upgrade dependency to Avro 1.5.3. (Alejandro Abdelnur via atm)
@@ -118,6 +123,9 @@ Release 0.23.1 - Unreleased
     HADOOP-7802. Hadoop scripts unconditionally source
     "$bin"/../libexec/hadoop-config.sh. (Bruno Mahé via tomwhite)
 
+    HADOOP-7858. Drop some info logging to DEBUG level in IPC,
+    metrics, and HTTP. (todd via eli)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -131,6 +139,8 @@ Release 0.23.1 - Unreleased
    HADOOP-6614. RunJar should provide more diags when it can't create
    a temp file. (Jonathan Hsieh via eli)
 
+   HADOOP-7859. TestViewFsHdfs.testgetFileLinkStatus is failing an assert. (eli)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES
@@ -1096,6 +1106,12 @@ Release 0.22.0 - Unreleased
 
     HADOOP-7786. Remove HDFS-specific config keys defined in FsConfig. (eli)
 
+    HADOOP-7358. Improve log levels when exceptions caught in RPC handler
+    (Todd Lipcon via shv)
+
+    HADOOP-7861. changes2html.pl generates links to HADOOP, HDFS, and MAPREDUCE
+    jiras. (shv)
+
   OPTIMIZATIONS
 
     HADOOP-6884. Add LOG.isDebugEnabled() guard for each LOG.debug(..).

+ 5 - 1
hadoop-common-project/hadoop-common/src/main/docs/changes/changes2html.pl

@@ -242,7 +242,11 @@ for my $rel (@releases) {
 
       $item =~ s:\s*(\([^)"]+?\))\s*$:<br />$1:;       # Separate attribution
       $item =~ s:\n{2,}:\n<p/>\n:g;                    # Keep paragraph breaks
-      $item =~ s{(?:${jira_url_prefix})?(HADOOP-\d+)}  # Link to JIRA
+      $item =~ s{(?:${jira_url_prefix})?(HADOOP-\d+)}  # Link to JIRA Common
+                {<a href="${jira_url_prefix}$1">$1</a>}g;
+      $item =~ s{(?:${jira_url_prefix})?(HDFS-\d+)}    # Link to JIRA Hdfs
+                {<a href="${jira_url_prefix}$1">$1</a>}g;
+      $item =~ s{(?:${jira_url_prefix})?(MAPREDUCE-\d+)}  # Link to JIRA MR
                 {<a href="${jira_url_prefix}$1">$1</a>}g;
       print "      <li>$item</li>\n";
     }

+ 2 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java

@@ -1099,18 +1099,10 @@ public final class FileContext {
    */
   private Path qualifySymlinkTarget(final AbstractFileSystem pathFS,
     Path pathWithLink, Path target) {
-    /* NB: makeQualified uses the target's scheme and authority, if
-     * specified, and the scheme and authority of pathFS, if not. If
-     * the path does have a scheme and authority we assert they match
-     * those of pathFS since resolve updates the file system of a path
-     * that contains links each time a link is encountered.
-     */
+    // NB: makeQualified uses the target's scheme and authority, if
+    // specified, and the scheme and authority of pathFS, if not.
     final String scheme = target.toUri().getScheme();
     final String auth   = target.toUri().getAuthority();
-    if (scheme != null && auth != null) {
-      assert scheme.equals(pathFS.getUri().getScheme());
-      assert auth.equals(pathFS.getUri().getAuthority());
-    }
     return (scheme == null && auth == null)
       ? target.makeQualified(pathFS.getUri(), pathWithLink.getParent())
       : target;

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java

@@ -644,12 +644,12 @@ public class HttpServer implements FilterContainer {
         while (true) {
           try {
             port = webServer.getConnectors()[0].getLocalPort();
-            LOG.info("Port returned by webServer.getConnectors()[0]." +
+            LOG.debug("Port returned by webServer.getConnectors()[0]." +
             		"getLocalPort() before open() is "+ port + 
             		". Opening the listener on " + oriPort);
             listener.open();
             port = listener.getLocalPort();
-            LOG.info("listener.getLocalPort() returned " + listener.getLocalPort() + 
+            LOG.debug("listener.getLocalPort() returned " + listener.getLocalPort() + 
                   " webServer.getConnectors()[0].getLocalPort() returned " +
                   webServer.getConnectors()[0].getLocalPort());
             //Workaround to handle the problem reported in HADOOP-4744

+ 12 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -1498,7 +1498,7 @@ public abstract class Server {
 
     @Override
     public void run() {
-      LOG.info(getName() + ": starting");
+      LOG.debug(getName() + ": starting");
       SERVER.set(Server.this);
       ByteArrayOutputStream buf = 
         new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE);
@@ -1536,7 +1536,16 @@ public abstract class Server {
                   );
             }
           } catch (Throwable e) {
-            LOG.info(getName() + ", call: " + call + ", error: ", e);
+            String logMsg = getName() + ", call " + call + ": error: " + e;
+            if (e instanceof RuntimeException || e instanceof Error) {
+              // These exception types indicate something is probably wrong
+              // on the server side, as opposed to just a normal exceptional
+              // result.
+              LOG.warn(logMsg, e);
+            } else {
+              LOG.info(logMsg, e);
+            }
+
             errorClass = e.getClass().getName();
             error = StringUtils.stringifyException(e);
             // Remove redundant error class name from the beginning of the stack trace
@@ -1571,7 +1580,7 @@ public abstract class Server {
           LOG.info(getName() + " caught an exception", e);
         }
       }
-      LOG.info(getName() + ": exiting");
+      LOG.debug(getName() + ": exiting");
     }
 
   }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java

@@ -389,7 +389,7 @@ public class WritableRpcEngine implements RpcEngine {
       }
       protocolImplMap.put(new ProtoNameVer(protocolName, version),
           new ProtoClassProtoImpl(protocolClass, protocolImpl)); 
-      LOG.info("Protocol Name = " + protocolName +  " version=" + version +
+      LOG.debug("Protocol Name = " + protocolName +  " version=" + version +
           " ProtocolImpl=" + protocolImpl.getClass().getName() + 
           " protocolClass=" + protocolClass.getName());
     }

+ 5 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java

@@ -241,7 +241,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
           injectedTags, period, config.subset(SOURCE_KEY));
     sources.put(name, sa);
     sa.start();
-    LOG.info("Registered source "+ name);
+    LOG.debug("Registered source "+ name);
   }
 
   @Override public synchronized <T extends MetricsSink>
@@ -405,8 +405,8 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   private synchronized void stopSources() {
     for (Entry<String, MetricsSourceAdapter> entry : sources.entrySet()) {
       MetricsSourceAdapter sa = entry.getValue();
-      LOG.info("Stopping metrics source "+ entry.getKey());
-      LOG.debug(sa.source().getClass());
+      LOG.debug("Stopping metrics source "+ entry.getKey() +
+          ": class=" + sa.source().getClass());
       sa.stop();
     }
     sysSource.stop();
@@ -416,8 +416,8 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   private synchronized void stopSinks() {
     for (Entry<String, MetricsSinkAdapter> entry : sinks.entrySet()) {
       MetricsSinkAdapter sa = entry.getValue();
-      LOG.info("Stopping metrics sink "+ entry.getKey());
-      LOG.debug(sa.sink().getClass());
+      LOG.debug("Stopping metrics sink "+ entry.getKey() +
+          ": class=" + sa.sink().getClass());
       sa.stop();
     }
     sinks.clear();

+ 107 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java

@@ -0,0 +1,107 @@
+/**
+ * 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.net;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * This is a base class for DNS to Switch mappings. <p/> It is not mandatory to
+ * derive {@link DNSToSwitchMapping} implementations from it, but it is strongly
+ * recommended, as it makes it easy for the Hadoop developers to add new methods
+ * to this base class that are automatically picked up by all implementations.
+ * <p/>
+ *
+ * This class does not extend the <code>Configured</code>
+ * base class, and should not be changed to do so, as it causes problems
+ * for subclasses. The constructor of the <code>Configured</code> calls
+ * the  {@link #setConf(Configuration)} method, which will call into the
+ * subclasses before they have been fully constructed.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class AbstractDNSToSwitchMapping
+    implements DNSToSwitchMapping, Configurable {
+
+  private Configuration conf;
+
+  /**
+   * Create an unconfigured instance
+   */
+  protected AbstractDNSToSwitchMapping() {
+  }
+
+  /**
+   * Create an instance, caching the configuration file.
+   * This constructor does not call {@link #setConf(Configuration)}; if
+   * a subclass extracts information in that method, it must call it explicitly.
+   * @param conf the configuration
+   */
+  protected AbstractDNSToSwitchMapping(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Predicate that indicates that the switch mapping is known to be
+   * single-switch. The base class returns false: it assumes all mappings are
+   * multi-rack. Subclasses may override this with methods that are more aware
+   * of their topologies.
+   *
+   * <p/>
+   *
+   * This method is used when parts of Hadoop need know whether to apply
+   * single rack vs multi-rack policies, such as during block placement.
+   * Such algorithms behave differently if they are on multi-switch systems.
+   * </p>
+   *
+   * @return true if the mapping thinks that it is on a single switch
+   */
+  public boolean isSingleSwitch() {
+    return false;
+  }
+
+  /**
+   * Query for a {@link DNSToSwitchMapping} instance being on a single
+   * switch.
+   * <p/>
+   * This predicate simply assumes that all mappings not derived from
+   * this class are multi-switch.
+   * @param mapping the mapping to query
+   * @return true if the base class says it is single switch, or the mapping
+   * is not derived from this class.
+   */
+  public static boolean isMappingSingleSwitch(DNSToSwitchMapping mapping) {
+    return mapping instanceof AbstractDNSToSwitchMapping
+        && ((AbstractDNSToSwitchMapping) mapping).isSingleSwitch();
+  }
+
+}

+ 16 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/CachedDNSToSwitchMapping.java

@@ -34,9 +34,13 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class CachedDNSToSwitchMapping implements DNSToSwitchMapping {
+public class CachedDNSToSwitchMapping extends AbstractDNSToSwitchMapping {
   private Map<String, String> cache = new ConcurrentHashMap<String, String>();
-  protected DNSToSwitchMapping rawMapping;
+
+  /**
+   * The uncached mapping
+   */
+  protected final DNSToSwitchMapping rawMapping;
 
   /**
    * cache a raw DNS mapping
@@ -118,4 +122,14 @@ public class CachedDNSToSwitchMapping implements DNSToSwitchMapping {
     return getCachedHosts(names);
 
   }
+
+  /**
+   * Delegate the switch topology query to the raw mapping, via
+   * {@link AbstractDNSToSwitchMapping#isMappingSingleSwitch(DNSToSwitchMapping)}
+   * @return true iff the raw mapper is considered single-switch.
+   */
+  @Override
+  public boolean isSingleSwitch() {
+    return isMappingSingleSwitch(rawMapping);
+  }
 }

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNSToSwitchMapping.java

@@ -40,6 +40,12 @@ public interface DNSToSwitchMapping {
    * Note the hostname/ip-address is not part of the returned path.
    * The network topology of the cluster would determine the number of
    * components in the network path.
+   * <p/>
+   *
+   * If a name cannot be resolved to a rack, the implementation
+   * should return {@link NetworkTopology#DEFAULT_RACK}. This
+   * is what the bundled implementations do, though it is not a formal requirement
+   *
    * @param names the list of hosts to resolve (can be empty)
    * @return list of resolved network paths.
    * If <i>names</i> is empty, the returned list is also empty

+ 98 - 57
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java

@@ -32,16 +32,21 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 
 /**
  * This class implements the {@link DNSToSwitchMapping} interface using a 
- * script configured via the {@link CommonConfigurationKeys#NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY}
+ * script configured via the
+ * {@link CommonConfigurationKeys#NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY} option.
+ * <p/>
+ * It contains a static class <code>RawScriptBasedMapping</code> that performs
+ * the work: reading the configuration parameters, executing any defined
+ * script, handling errors and such like. The outer
+ * class extends {@link CachedDNSToSwitchMapping} to cache the delegated
+ * queries.
+ * <p/>
+ * This DNS mapper's {@link #isSingleSwitch()} predicate returns
+ * true if and only if a script is defined.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public final class ScriptBasedMapping extends CachedDNSToSwitchMapping 
-implements Configurable
-{
-  public ScriptBasedMapping() {
-    super(new RawScriptBasedMapping());
-  }
+public final class ScriptBasedMapping extends CachedDNSToSwitchMapping {
 
   /**
    * Minimum number of arguments: {@value}
@@ -65,6 +70,18 @@ implements Configurable
   static final String SCRIPT_ARG_COUNT_KEY =
                      CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY ;
 
+  /**
+   * Create an instance with the default configuration.
+   * </p>
+   * Calling {@link #setConf(Configuration)} will trigger a
+   * re-evaluation of the configuration settings and so be used to
+   * set up the mapping script.
+   *
+   */
+  public ScriptBasedMapping() {
+    super(new RawScriptBasedMapping());
+  }
+
   /**
    * Create an instance from the given configuration
    * @param conf configuration
@@ -74,14 +91,31 @@ implements Configurable
     setConf(conf);
   }
 
+  /**
+   * Get the cached mapping and convert it to its real type
+   * @return the inner raw script mapping.
+   */
+  private RawScriptBasedMapping getRawMapping() {
+    return (RawScriptBasedMapping)rawMapping;
+  }
+
   @Override
   public Configuration getConf() {
-    return ((RawScriptBasedMapping)rawMapping).getConf();
+    return getRawMapping().getConf();
   }
 
+  /**
+   * {@inheritDoc}
+   * <p/>
+   * This will get called in the superclass constructor, so a check is needed
+   * to ensure that the raw mapping is defined before trying to relaying a null
+   * configuration.
+   * @param conf
+   */
   @Override
   public void setConf(Configuration conf) {
-    ((RawScriptBasedMapping)rawMapping).setConf(conf);
+    super.setConf(conf);
+    getRawMapping().setConf(conf);
   }
 
   /**
@@ -89,29 +123,26 @@ implements Configurable
    * by the superclass {@link CachedDNSToSwitchMapping}
    */
   private static final class RawScriptBasedMapping
-      implements DNSToSwitchMapping {
+      extends AbstractDNSToSwitchMapping {
     private String scriptName;
-    private Configuration conf;
     private int maxArgs; //max hostnames per call of the script
-    private static Log LOG =
+    private static final Log LOG =
         LogFactory.getLog(ScriptBasedMapping.class);
 
     /**
-     * Set the configuration and
-     * @param conf extract the configuration parameters of interest
+     * Set the configuration and extract the configuration parameters of interest
+     * @param conf the new configuration
      */
+    @Override
     public void setConf (Configuration conf) {
-      this.scriptName = conf.get(SCRIPT_FILENAME_KEY);
-      this.maxArgs = conf.getInt(SCRIPT_ARG_COUNT_KEY, DEFAULT_ARG_COUNT);
-      this.conf = conf;
-    }
-
-    /**
-     * Get the configuration
-     * @return the configuration
-     */
-    public Configuration getConf () {
-      return conf;
+      super.setConf(conf);
+      if (conf != null) {
+        scriptName = conf.get(SCRIPT_FILENAME_KEY);
+        maxArgs = conf.getInt(SCRIPT_ARG_COUNT_KEY, DEFAULT_ARG_COUNT);
+      } else {
+        scriptName = null;
+        maxArgs = 0;
+      }
     }
 
     /**
@@ -122,42 +153,42 @@ implements Configurable
 
     @Override
     public List<String> resolve(List<String> names) {
-    List <String> m = new ArrayList<String>(names.size());
-    
-    if (names.isEmpty()) {
-      return m;
-    }
+      List<String> m = new ArrayList<String>(names.size());
 
-    if (scriptName == null) {
-      for (int i = 0; i < names.size(); i++) {
-        m.add(NetworkTopology.DEFAULT_RACK);
+      if (names.isEmpty()) {
+        return m;
       }
-      return m;
-    }
-    
-    String output = runResolveCommand(names);
-    if (output != null) {
-      StringTokenizer allSwitchInfo = new StringTokenizer(output);
-      while (allSwitchInfo.hasMoreTokens()) {
-        String switchInfo = allSwitchInfo.nextToken();
-        m.add(switchInfo);
+
+      if (scriptName == null) {
+        for (String name : names) {
+          m.add(NetworkTopology.DEFAULT_RACK);
+        }
+        return m;
       }
-      
-      if (m.size() != names.size()) {
-        // invalid number of entries returned by the script
-        LOG.error("Script " + scriptName + " returned "
-            + Integer.toString(m.size()) + " values when "
-            + Integer.toString(names.size()) + " were expected.");
+
+      String output = runResolveCommand(names);
+      if (output != null) {
+        StringTokenizer allSwitchInfo = new StringTokenizer(output);
+        while (allSwitchInfo.hasMoreTokens()) {
+          String switchInfo = allSwitchInfo.nextToken();
+          m.add(switchInfo);
+        }
+
+        if (m.size() != names.size()) {
+          // invalid number of entries returned by the script
+          LOG.error("Script " + scriptName + " returned "
+              + Integer.toString(m.size()) + " values when "
+              + Integer.toString(names.size()) + " were expected.");
+          return null;
+        }
+      } else {
+        // an error occurred. return null to signify this.
+        // (exn was already logged in runResolveCommand)
         return null;
       }
-    } else {
-      // an error occurred. return null to signify this.
-      // (exn was already logged in runResolveCommand)
-      return null;
+
+      return m;
     }
-    
-    return m;
-  }
 
     /**
      * Build and execute the resolution command. The command is
@@ -195,10 +226,10 @@ implements Configurable
           dir = new File(userDir);
         }
         ShellCommandExecutor s = new ShellCommandExecutor(
-            cmdList.toArray(new String[0]), dir);
+            cmdList.toArray(new String[cmdList.size()]), dir);
         try {
           s.execute();
-          allOutput.append(s.getOutput() + " ");
+          allOutput.append(s.getOutput()).append(" ");
         } catch (Exception e) {
           LOG.warn("Exception: ", e);
           return null;
@@ -207,5 +238,15 @@ implements Configurable
       }
       return allOutput.toString();
     }
+
+    /**
+     * Declare that the mapper is single-switched if a script was not named
+     * in the configuration.
+     * @return true iff there is no script
+     */
+    @Override
+    public boolean isSingleSwitch() {
+      return scriptName == null;
+    }
   }
 }

+ 83 - 17
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/StaticMapping.java

@@ -17,34 +17,80 @@
  */
 package org.apache.hadoop.net;
 
-import java.util.*;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Implements the {@link DNSToSwitchMapping} via static mappings. Used
- * in testcases that simulate racks.
+ * in testcases that simulate racks, and in the
+ * {@link org.apache.hadoop.hdfs.MiniDFSCluster}
+ *
+ * A shared, static mapping is used; to reset it call {@link #resetMap()}.
  *
+ * When an instance of the class has its {@link #setConf(Configuration)}
+ * method called, nodes listed in the configuration will be added to the map.
+ * These do not get removed when the instance is garbage collected.
  */
-public class StaticMapping extends Configured implements DNSToSwitchMapping {
-  public void setconf(Configuration conf) {
-    String[] mappings = conf.getStrings("hadoop.configured.node.mapping");
-    if (mappings != null) {
-      for (int i = 0; i < mappings.length; i++) {
-        String str = mappings[i];
-        String host = str.substring(0, str.indexOf('='));
-        String rack = str.substring(str.indexOf('=') + 1);
-        addNodeToRack(host, rack);
+public class StaticMapping extends AbstractDNSToSwitchMapping  {
+
+  /**
+   * key to define the node mapping as a comma-delimited list of host=rack
+   * mappings, e.g. <code>host1=r1,host2=r1,host3=r2</code>.
+   * </p>
+   * <b>Important: </b>spaces not trimmed and are considered significant.
+   */
+  public static final String KEY_HADOOP_CONFIGURED_NODE_MAPPING =
+      "hadoop.configured.node.mapping";
+
+  /**
+   * Configure the mapping by extracting any mappings defined in the
+   * {@link #KEY_HADOOP_CONFIGURED_NODE_MAPPING} field
+   * @param conf new configuration
+   */
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    if (conf != null) {
+      String[] mappings = conf.getStrings(KEY_HADOOP_CONFIGURED_NODE_MAPPING);
+      if (mappings != null) {
+        for (String str : mappings) {
+          String host = str.substring(0, str.indexOf('='));
+          String rack = str.substring(str.indexOf('=') + 1);
+          addNodeToRack(host, rack);
+        }
       }
     }
   }
+
+  /**
+   * retained lower case setter for compatibility reasons; relays to
+   * {@link #setConf(Configuration)}
+   * @param conf new configuration
+   */
+  public void setconf(Configuration conf) {
+    setConf(conf);
+  }
+
   /* Only one instance per JVM */
-  private static Map<String, String> nameToRackMap = new HashMap<String, String>();
-  
-  static synchronized public void addNodeToRack(String name, String rackId) {
-    nameToRackMap.put(name, rackId);
+  private static final Map<String, String> nameToRackMap = new HashMap<String, String>();
+
+  /**
+   * Add a node to the static map. The moment any entry is added to the map,
+   * the map goes multi-rack.
+   * @param name node name
+   * @param rackId rack ID
+   */
+  public static void addNodeToRack(String name, String rackId) {
+    synchronized (nameToRackMap) {
+      nameToRackMap.put(name, rackId);
+    }
   }
+
+  @Override
   public List<String> resolve(List<String> names) {
     List<String> m = new ArrayList<String>();
     synchronized (nameToRackMap) {
@@ -59,4 +105,24 @@ public class StaticMapping extends Configured implements DNSToSwitchMapping {
       return m;
     }
   }
+
+  /**
+   * This mapping is only single switch if the map is empty
+   * @return the current switching status
+   */
+  @Override
+  public boolean isSingleSwitch() {
+    synchronized (nameToRackMap) {
+      return nameToRackMap.isEmpty();
+    }
+  }
+
+  /**
+   * Clear the map and revert to being a single switch
+   */
+  public static void resetMap() {
+    synchronized (nameToRackMap) {
+      nameToRackMap.clear();
+    }
+  }
 }

+ 41 - 12
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java

@@ -23,30 +23,59 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 
 import junit.framework.TestCase;
+import org.junit.Test;
 
 public class TestScriptBasedMapping extends TestCase {
 
-  private ScriptBasedMapping mapping;
-  private Configuration conf;
-  private List<String> names;
+
   
   public TestScriptBasedMapping() {
-    mapping = new ScriptBasedMapping();
-
-    conf = new Configuration();
-    conf.setInt(ScriptBasedMapping.SCRIPT_ARG_COUNT_KEY,
-        ScriptBasedMapping.MIN_ALLOWABLE_ARGS - 1);
-    conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename");
 
-    mapping.setConf(conf);    
   }
 
+  @Test
   public void testNoArgsMeansNoResult() {
-    names = new ArrayList<String>();
+    Configuration conf = new Configuration();
+    conf.setInt(ScriptBasedMapping.SCRIPT_ARG_COUNT_KEY,
+                ScriptBasedMapping.MIN_ALLOWABLE_ARGS - 1);
+    conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename");
+    conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename");
+    ScriptBasedMapping mapping = createMapping(conf);
+    List<String> names = new ArrayList<String>();
     names.add("some.machine.name");
     names.add("other.machine.name");
     List<String> result = mapping.resolve(names);
-    assertNull(result);
+    assertNull("Expected an empty list", result);
   }
 
+  @Test
+  public void testNoFilenameMeansSingleSwitch() throws Throwable {
+    Configuration conf = new Configuration();
+    ScriptBasedMapping mapping = createMapping(conf);
+    assertTrue("Expected to be single switch", mapping.isSingleSwitch());
+    assertTrue("Expected to be single switch",
+               AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
+  }
+
+  @Test
+  public void testFilenameMeansMultiSwitch() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename");
+    ScriptBasedMapping mapping = createMapping(conf);
+    assertFalse("Expected to be multi switch", mapping.isSingleSwitch());
+    mapping.setConf(new Configuration());
+    assertTrue("Expected to be single switch", mapping.isSingleSwitch());
+  }
+
+  @Test
+  public void testNullConfig() throws Throwable {
+    ScriptBasedMapping mapping = createMapping(null);
+    assertTrue("Expected to be single switch", mapping.isSingleSwitch());
+
+  }
+  private ScriptBasedMapping createMapping(Configuration conf) {
+    ScriptBasedMapping mapping = new ScriptBasedMapping();
+    mapping.setConf(conf);
+    return mapping;
+  }
 }

+ 103 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java

@@ -0,0 +1,103 @@
+/**
+ * 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.net;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test the static mapping class.
+ * Because the map is actually static, this map needs to be reset for every test
+ */
+public class TestStaticMapping extends Assert {
+
+  /**
+   * Reset the map then create a new instance of the {@link StaticMapping}
+   * class
+   * @return a new instance
+   */
+  private StaticMapping newInstance() {
+    StaticMapping.resetMap();
+    return new StaticMapping();
+  }
+
+  @Test
+  public void testStaticIsSingleSwitch() throws Throwable {
+    StaticMapping mapping = newInstance();
+    assertTrue("Empty maps are not single switch", mapping.isSingleSwitch());
+  }
+
+
+  @Test
+  public void testCachingRelaysQueries() throws Throwable {
+    StaticMapping staticMapping = newInstance();
+    CachedDNSToSwitchMapping mapping =
+        new CachedDNSToSwitchMapping(staticMapping);
+    assertTrue("Expected single switch", mapping.isSingleSwitch());
+    StaticMapping.addNodeToRack("n1", "r1");
+    assertFalse("Expected to be multi switch",
+                mapping.isSingleSwitch());
+  }
+
+  @Test
+  public void testAddResolveNodes() throws Throwable {
+    StaticMapping mapping = newInstance();
+    StaticMapping.addNodeToRack("n1", "r1");
+    List<String> l1 = new ArrayList<String>(2);
+    l1.add("n1");
+    l1.add("unknown");
+    List<String> mappings = mapping.resolve(l1);
+    assertEquals(2, mappings.size());
+    assertEquals("r1", mappings.get(0));
+    assertEquals(NetworkTopology.DEFAULT_RACK, mappings.get(1));
+    assertFalse("Mapping is still single switch", mapping.isSingleSwitch());
+  }
+
+  @Test
+  public void testReadNodesFromConfig() throws Throwable {
+    StaticMapping mapping = newInstance();
+    Configuration conf = new Configuration();
+    conf.set(StaticMapping.KEY_HADOOP_CONFIGURED_NODE_MAPPING, "n1=r1,n2=r2");
+    mapping.setConf(conf);
+    List<String> l1 = new ArrayList<String>(3);
+    l1.add("n1");
+    l1.add("unknown");
+    l1.add("n2");
+    List<String> mappings = mapping.resolve(l1);
+    assertEquals(3, mappings.size());
+    assertEquals("r1", mappings.get(0));
+    assertEquals(NetworkTopology.DEFAULT_RACK, mappings.get(1));
+    assertEquals("r2", mappings.get(2));
+    assertFalse("Expected to be multi switch",
+                AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
+  }
+
+  @Test
+  public void testNullConfiguration() throws Throwable {
+    StaticMapping mapping = newInstance();
+    mapping.setConf(null);
+    assertTrue("Null maps is not single switch", mapping.isSingleSwitch());
+    assertTrue("Expected to be single switch",
+               AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
+  }
+}

+ 53 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.net;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Test some other details of the switch mapping
+ */
+public class TestSwitchMapping extends Assert {
+
+  @Test
+  public void testStandaloneClassesAssumedMultiswitch() throws Throwable {
+    DNSToSwitchMapping mapping = new StandaloneSwitchMapping();
+    assertFalse("Expected to be multi switch",
+                AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
+  }
+
+
+  @Test
+  public void testCachingRelays() throws Throwable {
+    CachedDNSToSwitchMapping mapping =
+        new CachedDNSToSwitchMapping(new StandaloneSwitchMapping());
+    assertFalse("Expected to be multi switch",
+                mapping.isSingleSwitch());
+  }
+
+  private static class StandaloneSwitchMapping implements DNSToSwitchMapping {
+    @Override
+    public List<String> resolve(List<String> names) {
+      return names;
+    }
+  }
+}

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -11,6 +11,8 @@ Trunk (unreleased changes)
 
     HDFS-2520. Add protobuf service for InterDatanodeProtocol. (suresh)
 
+    HDFS-2519. Add protobuf service for DatanodeProtocol. (suresh)
+
   IMPROVEMENTS
 
     HADOOP-7524 Change RPC to allow multiple protocols including multuple 
@@ -122,6 +124,9 @@ Release 0.23.1 - UNRELEASED
 
   NEW FEATURES
 
+    HDFS-2316. [umbrella] webhdfs: a complete FileSystem implementation for 
+    accessing HDFS over HTTP (szetszwo)
+
   IMPROVEMENTS
     HDFS-2560. Refactor BPOfferService to be a static inner class (todd)
 
@@ -151,6 +156,10 @@ Release 0.23.1 - UNRELEASED
 
     HDFS-2566. Move BPOfferService to be a non-inner class. (todd)
 
+    HDFS-2552. Add Forrest doc for WebHDFS REST API.  (szetszwo)
+
+    HDFS-2587. Add apt doc for WebHDFS REST API.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-2130. Switch default checksum to CRC32C. (todd)
@@ -177,6 +186,10 @@ Release 0.23.1 - UNRELEASED
     trying to browse DFS via web UI. (harsh via eli)
 
     HDFS-2575. DFSTestUtil may create empty files (todd)
+
+    HDFS-2588. hdfs jsp pages missing DOCTYPE. (Dave Vronay via mattf)
+
+    HDFS-2590. Fix the missing links in the WebHDFS forrest doc.  (szetszwo)
     
 Release 0.23.0 - 2011-11-01 
 

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/docs/changes/changes2html.pl

@@ -242,7 +242,11 @@ for my $rel (@releases) {
 
       $item =~ s:\s*(\([^)"]+?\))\s*$:<br />$1:;       # Separate attribution
       $item =~ s:\n{2,}:\n<p/>\n:g;                    # Keep paragraph breaks
-      $item =~ s{(?:${jira_url_prefix})?(HADOOP-\d+)}  # Link to JIRA
+      $item =~ s{(?:${jira_url_prefix})?(HADOOP-\d+)}  # Link to JIRA Common
+                {<a href="${jira_url_prefix}$1">$1</a>}g;
+      $item =~ s{(?:${jira_url_prefix})?(HDFS-\d+)}    # Link to JIRA Hdfs
+                {<a href="${jira_url_prefix}$1">$1</a>}g;
+      $item =~ s{(?:${jira_url_prefix})?(MAPREDUCE-\d+)}  # Link to JIRA MR
                 {<a href="${jira_url_prefix}$1">$1</a>}g;
       print "      <li>$item</li>\n";
     }

+ 27 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/docs/src/documentation/content/xdocs/site.xml

@@ -43,6 +43,7 @@ See http://forrest.apache.org/docs/linking.html for more info.
       <hdfs_SLG        		label="Synthetic Load Generator"  href="SLG_user_guide.html" />
       <hdfs_imageviewer	label="Offline Image Viewer"	href="hdfs_imageviewer.html" />
       <hdfs_editsviewer	label="Offline Edits Viewer"	href="hdfs_editsviewer.html" />
+      <webhdfs label="WebHDFS REST API" href="webhdfs.html" />
       <hftp 			    label="HFTP" href="hftp.html"/>
       <faultinject_framework label="Fault Injection"  href="faultinject_framework.html" /> 
       <hdfs_libhdfs   		label="C API libhdfs" href="libhdfs.html" /> 
@@ -119,8 +120,33 @@ See http://forrest.apache.org/docs/linking.html for more info.
               </distributedcache>  
             </filecache>
             <fs href="fs/">
-              <filesystem href="FileSystem.html" />
+              <FileStatus href="FileStatus.html" />
+              <Path href="Path.html" />
+
+              <filesystem href="FileSystem.html">
+                <open href="#open(org.apache.hadoop.fs.Path,%20int)" />
+                <getFileStatus href="#getFileStatus(org.apache.hadoop.fs.Path)" />
+                <listStatus href="#listStatus(org.apache.hadoop.fs.Path)" />
+                <getContentSummary href="#getContentSummary(org.apache.hadoop.fs.Path)" />
+                <getFileChecksum href="#getFileChecksum(org.apache.hadoop.fs.Path)" />
+                <getHomeDirectory href="#getHomeDirectory()" />
+                <getDelegationToken href="#getDelegationToken(org.apache.hadoop.io.Text)" />
+
+                <create href="#create(org.apache.hadoop.fs.Path,%20org.apache.hadoop.fs.permission.FsPermission,%20boolean,%20int,%20short,%20long,%20org.apache.hadoop.util.Progressable)" />
+                <mkdirs href="#mkdirs(org.apache.hadoop.fs.Path,%20org.apache.hadoop.fs.permission.FsPermission)" />
+                <rename href="#rename(org.apache.hadoop.fs.Path,%20org.apache.hadoop.fs.Path,%20org.apache.hadoop.fs.Options.Rename...)" />
+                <setReplication href="#setReplication(org.apache.hadoop.fs.Path,%20short)" />
+                <setOwner href="#setOwner(org.apache.hadoop.fs.Path,%20java.lang.String,%20java.lang.String)" />
+                <setPermission href="#setPermission(org.apache.hadoop.fs.Path,%20org.apache.hadoop.fs.permission.FsPermission)" />
+                <setTimes href="#setTimes(org.apache.hadoop.fs.Path,%20long,%20long)" />
+
+                <append href="#append(org.apache.hadoop.fs.Path,%20int,%20org.apache.hadoop.util.Progressable)" />
+                <delete href="#delete(org.apache.hadoop.fs.Path,%20boolean)" />
+              </filesystem>
             </fs>
+
+
+
             <io href="io/">
               <closeable href="Closeable.html">
                 <close href="#close()" />

+ 1555 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/docs/src/documentation/content/xdocs/webhdfs.xml

@@ -0,0 +1,1555 @@
+<?xml version="1.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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document>
+  <header>
+    <title>WebHDFS REST API</title>
+  </header>
+
+  <body>
+    <section>
+      <title>Document Conventions</title>
+<table>
+<tr><td><code>Monospaced</code></td><td>Used for commands, HTTP request and responses and code blocks.</td></tr>
+<tr><td><code>&lt;Monospaced&gt;</code></td><td>User entered values.</td></tr>
+<tr><td><code>[Monospaced]</code></td><td>Optional values.  When the value is not specified, the default value is used.</td></tr>
+<tr><td><em>Italics</em></td><td>Important phrases and words.</td></tr>
+</table>
+    </section>
+<!-- ***************************************************************************** -->
+    <section>
+      <title>Introduction</title>
+<p>
+  The HTTP REST API supports the complete FileSystem interface for HDFS.
+  The operations and the corresponding FileSystem methods are shown in the next section.
+  The Section <a href="#ParameterDictionary">HTTP Query Parameter Dictionary</a> specifies the parameter details
+  such as the defaults and the valid values.
+</p>
+      <section id="Operations">
+        <title>Operations</title>
+<ul>
+  <li>HTTP GET
+  <ul>
+    <li><a href="#OPEN"><code>OPEN</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/open">FileSystem.open</a>)
+    </li>
+    <li><a href="#GETFILESTATUS"><code>GETFILESTATUS</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/getFileStatus">FileSystem.getFileStatus</a>)
+    </li>
+    <li><a href="#LISTSTATUS"><code>LISTSTATUS</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/listStatus">FileSystem.listStatus</a>)
+    </li>
+    <li><a href="#GETCONTENTSUMMARY"><code>GETCONTENTSUMMARY</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/getContentSummary">FileSystem.getContentSummary</a>)
+    </li>
+    <li><a href="#GETFILECHECKSUM"><code>GETFILECHECKSUM</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/getFileChecksum">FileSystem.getFileChecksum</a>)
+    </li>
+    <li><a href="#GETHOMEDIRECTORY"><code>GETHOMEDIRECTORY</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/getHomeDirectory">FileSystem.getHomeDirectory</a>)
+    </li>
+    <li><a href="#GETDELEGATIONTOKEN"><code>GETDELEGATIONTOKEN</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/getDelegationToken">FileSystem.getDelegationToken</a>)
+    </li>
+  </ul></li>
+  <li>HTTP PUT
+  <ul>
+    <li><a href="#CREATE"><code>CREATE</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/create">FileSystem.create</a>)
+    </li>
+    <li><a href="#MKDIRS"><code>MKDIRS</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/mkdirs">FileSystem.mkdirs</a>)
+    </li>
+    <li><a href="#RENAME"><code>RENAME</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/rename">FileSystem.rename</a>)
+    </li>
+    <li><a href="#SETREPLICATION"><code>SETREPLICATION</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/setReplication">FileSystem.setReplication</a>)
+    </li>
+    <li><a href="#SETOWNER"><code>SETOWNER</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/setOwner">FileSystem.setOwner</a>)
+    </li>
+    <li><a href="#SETPERMISSION"><code>SETPERMISSION</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/setPermission">FileSystem.setPermission</a>)
+    </li>
+    <li><a href="#SETTIMES"><code>SETTIMES</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/setTimes">FileSystem.setTimes</a>)
+    </li>
+    <li><a href="#RENEWDELEGATIONTOKEN"><code>RENEWDELEGATIONTOKEN</code></a>
+        (see DistributedFileSystem.renewDelegationToken)
+    </li>
+    <li><a href="#CANCELDELEGATIONTOKEN"><code>CANCELDELEGATIONTOKEN</code></a>
+        (see DistributedFileSystem.cancelDelegationToken)
+    </li>
+  </ul></li>
+  <li>HTTP POST
+  <ul>
+    <li><a href="#APPEND"><code>APPEND</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/append">FileSystem.append</a>)
+    </li>
+  </ul></li>
+  <li>HTTP DELETE
+  <ul>
+    <li><a href="#DELETE"><code>DELETE</code></a>
+        (see <a href="ext:api/org/apache/hadoop/fs/filesystem/delete">FileSystem.delete</a>)
+    </li>
+  </ul></li>
+</ul>
+
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="FsURIvsHTTP_URL">
+        <title>FileSystem URIs vs HTTP URLs</title>
+<p>
+  The FileSystem scheme of WebHDFS is "<code>webhdfs://</code>".
+  A WebHDFS FileSystem URI has the following format.
+</p>
+<source>
+  webhdfs://&lt;HOST&gt;:&lt;HTTP_PORT&gt;/&lt;PATH&gt;
+</source>
+<p>
+  The above WebHDFS URI corresponds to the below HDFS URI.
+</p>
+<source>
+  hdfs://&lt;HOST&gt;:&lt;RPC_PORT&gt;/&lt;PATH&gt;
+</source>
+<p>
+  In the REST API, the prefix "<code>/webhdfs/v1</code>" is inserted in the path and a query is appended at the end.
+  Therefore, the corresponding HTTP URL has the following format.
+</p>
+<source>
+  http://&lt;HOST&gt;:&lt;HTTP_PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=...
+</source>
+      </section>
+    </section>
+<!-- ***************************************************************************** -->
+    <section id="Authentication">
+      <title>Authentication</title>
+<p>
+  When security is <em>off</em>, the authenticated user is the username specified in the <code>user.name</code> query parameter.
+  If the <code>user.name</code> parameter is not set,
+  the server may either set the authenticated user to a default web user, if there is any, or return an error response.
+</p>
+<p>
+  When security is <em>on</em>, authentication is performed by either Hadoop delegation token or Kerberos SPNEGO.
+  If a token is set in the <code>delegation</code> query parameter, the authenticated user is the user encoded in the token.
+  If the <code>delegation</code> parameter is not set, the user is authenticated by Kerberos SPNEGO.
+</p>
+<p>
+Below are examples using the <code>curl</code> command tool.
+</p>
+<ol>
+  <li>
+    Authentication when security is off:
+    <source>
+curl -i "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?[user.name=&lt;USER&gt;&amp;]op=..."
+    </source>
+  </li><li>
+    Authentication using Kerberos SPNEGO when security is on:
+    <source>
+curl -i --negotiate -u : "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=..."
+    </source>
+  </li><li>
+    Authentication using Hadoop delegation token when security is on:
+    <source>
+curl -i "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?delegation=&lt;TOKEN&gt;&amp;op=..."
+    </source>
+  </li>
+</ol>
+    </section>
+<!-- ***************************************************************************** -->
+    <section id="ProxyUsers">
+      <title>Proxy Users</title>
+<p>
+  When the proxy user feature is enabled, a proxy user <em>P</em> may submit a request on behalf of another user <em>U</em>.
+  The username of <em>U</em> must be specified in the <code>doas</code> query parameter unless a delegation token is presented in authentication.
+  In such case, the information of both users <em>P</em> and <em>U</em> must be encoded in the delegation token.
+</p>
+<ol>
+  <li>
+    A proxy request when security is off:
+    <source>
+curl -i "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?[user.name=&lt;USER&gt;&amp;]doas=&lt;USER&gt;&amp;op=..."
+    </source>
+  </li><li>
+    A proxy request using Kerberos SPNEGO when security is on:
+    <source>
+curl -i --negotiate -u : "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?doas=&lt;USER&gt;&amp;op=..."
+    </source>
+  </li><li>
+    A proxy request using Hadoop delegation token when security is on:
+    <source>
+curl -i "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?delegation=&lt;TOKEN&gt;&amp;op=..."
+    </source>
+  </li>
+</ol>
+    </section>
+<!-- ***************************************************************************** -->
+<!-- ***************************************************************************** -->
+    <section>
+      <title>File and Directory Operations</title>
+      <section id="CREATE">
+        <title>Create and Write to a File</title>
+<ul>
+  <li>Step 1: Submit a HTTP PUT request without automatically following redirects and without sending the file data.
+    <source>
+curl -i -X PUT "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=CREATE
+                    [&amp;overwrite=&lt;true|false&gt;][&amp;blocksize=&lt;LONG&gt;][&amp;replication=&lt;SHORT&gt;]
+                    [&amp;permission=&lt;OCTAL&gt;][&amp;buffersize=&lt;INT&gt;]"
+    </source>
+The request is redirected to a datanode where the file data is to be written:
+    <source>
+HTTP/1.1 307 TEMPORARY_REDIRECT
+Location: http://&lt;DATANODE&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=CREATE...
+Content-Length: 0
+    </source>
+  </li>
+  <li>Step 2: Submit another HTTP PUT request using the URL in the <code>Location</code> header with the file data to be written.
+    <source>
+curl -i -X PUT -T &lt;LOCAL_FILE&gt; "http://&lt;DATANODE&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=CREATE..."
+    </source>
+The client receives a <code>201 Created</code> response with zero content length
+and the WebHDFS URI of the file in the <code>Location</code> header:
+    <source>
+HTTP/1.1 201 Created
+Location: webhdfs://&lt;HOST&gt;:&lt;PORT&gt;/&lt;PATH&gt;
+Content-Length: 0
+    </source>
+  </li>
+</ul>
+<p>
+  <strong>Note</strong> that the reason of having two-step create/append is
+  for preventing clients to send out data before the redirect.
+  This issue is addressed by the "<code>Expect: 100-continue</code>" header in HTTP/1.1;
+  see <a href="http://www.w3.org/Protocols/rfc2616/rfc2616-sec8.html#sec8.2.3">RFC 2616, Section 8.2.3</a>.
+  Unfortunately, there are software library bugs (e.g. Jetty 6 HTTP server and Java 6 HTTP client),
+  which do not correctly implement "<code>Expect: 100-continue</code>".
+  The two-step create/append is a temporary workaround for the software library bugs.
+</p>
+<p>
+  See also:
+  <a href="#overwrite"><code>overwrite</code></a>,
+  <a href="#blocksize"><code>blocksize</code></a>,
+  <a href="#replication"><code>replication</code></a>,
+  <a href="#permission"><code>permission</code></a>,
+  <a href="#buffersize"><code>buffersize</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/create">FileSystem.create</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="APPEND">
+        <title>Append to a File</title>
+<ul>
+  <li>Step 1: Submit a HTTP POST request without automatically following redirects and without sending the file data.
+    <source>
+curl -i -X POST "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=APPEND[&amp;buffersize=&lt;INT&gt;]"
+    </source>
+The request is redirected to a datanode where the file data is to be appended:
+    <source>
+HTTP/1.1 307 TEMPORARY_REDIRECT
+Location: http://&lt;DATANODE&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=APPEND...
+Content-Length: 0
+    </source>
+  </li>
+  <li>Step 2: Submit another HTTP POST request using the URL in the <code>Location</code> header with the file data to be appended.
+    <source>
+curl -i -X POST -T &lt;LOCAL_FILE&gt; "http://&lt;DATANODE&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=APPEND..."
+    </source>
+The client receives a response with zero content length:
+    <source>
+HTTP/1.1 200 OK
+Content-Length: 0
+    </source>
+  </li>
+</ul>
+<p><em>
+  See the note in the previous section for the description of why this operation requires two steps.
+</em></p>
+<p>
+  See also:
+  <a href="#buffersize"><code>buffersize</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/append">FileSystem.append</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="OPEN">
+        <title>Open and Read a File</title>
+<ul>
+  <li>Submit a HTTP GET request with automatically following redirects.
+    <source>
+curl -i -L "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=OPEN
+                    [&amp;offset=&lt;LONG&gt;][&amp;length=&lt;LONG&gt;][&amp;buffersize=&lt;INT&gt;]"
+    </source>
+The request is redirected to a datanode where the file data can be read:
+    <source>
+HTTP/1.1 307 TEMPORARY_REDIRECT
+Location: http://&lt;DATANODE&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=OPEN...
+Content-Length: 0
+    </source>
+The client follows the redirect to the datanode and receives the file data:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/octet-stream
+Content-Length: 22
+
+Hello, webhdfs user!
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#offset"><code>offset</code></a>,
+  <a href="#length"><code>length</code></a>,
+  <a href="#buffersize"><code>buffersize</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/open">FileSystem.open</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="MKDIRS">
+        <title>Make a Directory</title>
+<ul>
+  <li>Submit a HTTP PUT request.
+    <source>
+curl -i -X PUT "http://&lt;HOST&gt;:&lt;PORT&gt;/&lt;PATH&gt;?op=MKDIRS[&amp;permission=&lt;OCTAL&gt;]"
+    </source>
+The client receives a response with a <a href="#boolean"><code>boolean</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"boolean": true}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#permission"><code>permission</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/mkdirs">FileSystem.mkdirs</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="RENAME">
+        <title>Rename a File/Directory</title>
+<ul>
+  <li>Submit a HTTP PUT request.
+    <source>
+curl -i -X PUT "&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=RENAME&amp;destination=&lt;PATH&gt;"
+    </source>
+The client receives a response with a <a href="#boolean"><code>boolean</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"boolean": true}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#destination"><code>destination</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/rename">FileSystem.rename</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="DELETE">
+        <title>Delete a File/Directory</title>
+<ul>
+  <li>Submit a HTTP DELETE request.
+    <source>
+curl -i -X DELETE "http://&lt;host&gt;:&lt;port&gt;/webhdfs/v1/&lt;path&gt;?op=DELETE
+                              [&amp;recursive=&lt;true|false&gt;]"
+    </source>
+The client receives a response with a <a href="#boolean"><code>boolean</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"boolean": true}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#recursive"><code>recursive</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/delete">FileSystem.delete</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="GETFILESTATUS">
+        <title>Status of a File/Directory</title>
+<ul>
+  <li>Submit a HTTP GET request.
+    <source>
+curl -i  "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=GETFILESTATUS"
+    </source>
+The client receives a response with a <a href="#FileStatus"><code>FileStatus</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "FileStatus":
+  {
+    "accessTime"      : 0,
+    "blockSize"       : 0,
+    "group"           : "supergroup",
+    "length"          : 0,             //in bytes, zero for directories
+    "modificationTime": 1320173277227,
+    "owner"           : "webuser",
+    "pathSuffix"      : "",
+    "permission"      : "777",
+    "replication"     : 0,
+    "type"            : "DIRECTORY"    //enum {FILE, DIRECTORY, SYMLINK}
+  }
+}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/getFileStatus">FileSystem.getFileStatus</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="LISTSTATUS">
+        <title>List a Directory</title>
+<ul>
+  <li>Submit a HTTP GET request.
+    <source>
+curl -i  "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=LISTSTATUS"
+    </source>
+The client receives a response with a <a href="#FileStatuses"><code>FileStatuses</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Content-Length: 427
+
+{
+  "FileStatuses":
+  {
+    "FileStatus":
+    [
+      {
+        "accessTime"      : 1320171722771,
+        "blockSize"       : 33554432,
+        "group"           : "supergroup",
+        "length"          : 24930,
+        "modificationTime": 1320171722771,
+        "owner"           : "webuser",
+        "pathSuffix"      : "a.patch",
+        "permission"      : "644",
+        "replication"     : 1,
+        "type"            : "FILE"
+      },
+      {
+        "accessTime"      : 0,
+        "blockSize"       : 0,
+        "group"           : "supergroup",
+        "length"          : 0,
+        "modificationTime": 1320895981256,
+        "owner"           : "szetszwo",
+        "pathSuffix"      : "bar",
+        "permission"      : "711",
+        "replication"     : 0,
+        "type"            : "DIRECTORY"
+      },
+      ...
+    ]
+  }
+}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/listStatus">FileSystem.listStatus</a>
+</p>
+      </section>
+    </section>
+<!-- ***************************************************************************** -->
+<!-- ***************************************************************************** -->
+    <section>
+      <title>Other File System Operations</title>
+      <section id="GETCONTENTSUMMARY">
+        <title>Get Content Summary of a Directory</title>
+<ul>
+  <li>Submit a HTTP GET request.
+    <source>
+curl -i "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=GETCONTENTSUMMARY"
+    </source>
+The client receives a response with a <a href="#ContentSummary"><code>ContentSummary</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "ContentSummary":
+  {
+    "directoryCount": 2,
+    "fileCount"     : 1,
+    "length"        : 24930,
+    "quota"         : -1,
+    "spaceConsumed" : 24930,
+    "spaceQuota"    : -1
+  }
+}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/getContentSummary">FileSystem.getContentSummary</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="GETFILECHECKSUM">
+        <title>Get File Checksum</title>
+<ul>
+  <li>Submit a HTTP GET request.
+    <source>
+curl -i "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=GETFILECHECKSUM"
+    </source>
+The request is redirected to a datanode:
+    <source>
+HTTP/1.1 307 TEMPORARY_REDIRECT
+Location: http://&lt;DATANODE&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=GETFILECHECKSUM...
+Content-Length: 0
+    </source>
+The client follows the redirect to the datanode and receives a <a href="#FileChecksum"><code>FileChecksum</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "FileChecksum":
+  {
+    "algorithm": "MD5-of-1MD5-of-512CRC32",
+    "bytes"    : "eadb10de24aa315748930df6e185c0d ...",
+    "length"   : 28
+  }
+}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/getFileChecksum">FileSystem.getFileChecksum</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="GETHOMEDIRECTORY">
+        <title>Get Home Directory</title>
+<ul>
+  <li>Submit a HTTP GET request.
+    <source>
+curl -i "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/?op=GETHOMEDIRECTORY"
+    </source>
+The client receives a response with a <a href="#Path"><code>Path</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"Path": "/user/szetszwo"}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/getHomeDirectory">FileSystem.getHomeDirectory</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="SETPERMISSION">
+        <title>Set Permission</title>
+<ul>
+  <li>Submit a HTTP PUT request.
+    <source>
+curl -i -X PUT "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=SETPERMISSION
+                              [&amp;permission=&lt;OCTAL&gt;]"
+    </source>
+The client receives a response with zero content length:
+    <source>
+HTTP/1.1 200 OK
+Content-Length: 0
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#permission"><code>permission</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/setPermission">FileSystem.setPermission</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="SETOWNER">
+        <title>Set Owner</title>
+<ul>
+  <li>Submit a HTTP PUT request.
+    <source>
+curl -i -X PUT "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=SETOWNER
+                              [&amp;owner=&lt;USER&gt;][&amp;group=&lt;GROUP&gt;]"
+    </source>
+The client receives a response with zero content length:
+    <source>
+HTTP/1.1 200 OK
+Content-Length: 0
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#owner"><code>owner</code></a>,
+  <a href="#group"><code>group</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/setOwner">FileSystem.setOwner</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="SETREPLICATION">
+        <title>Set Replication Factor</title>
+<ul>
+  <li>Submit a HTTP PUT request.
+    <source>
+curl -i -X PUT "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=SETREPLICATION
+                              [&amp;replication=&lt;SHORT&gt;]"
+    </source>
+The client receives a response with a <a href="#boolean"><code>boolean</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"boolean": true}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#replication"><code>replication</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/setReplication">FileSystem.setReplication</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="SETTIMES">
+        <title>Set Access or Modification Time</title>
+<ul>
+  <li>Submit a HTTP PUT request.
+    <source>
+curl -i -X PUT "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/&lt;PATH&gt;?op=SETTIMES
+                              [&amp;modificationtime=&lt;TIME&gt;][&amp;accesstime=&lt;TIME&gt;]"
+    </source>
+The client receives a response with zero content length:
+    <source>
+HTTP/1.1 200 OK
+Content-Length: 0
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#modificationtime"><code>modificationtime</code></a>,
+  <a href="#accesstime"><code>accesstime</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/setTimes">FileSystem.setTimes</a>
+</p>
+      </section>
+    </section>
+<!-- ***************************************************************************** -->
+<!-- ***************************************************************************** -->
+    <section>
+      <title>Delegation Token Operations</title>
+      <section id="GETDELEGATIONTOKEN">
+        <title>Get Delegation Token</title>
+<ul>
+  <li>Submit a HTTP GET request.
+    <source>
+curl -i "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/?op=GETDELEGATIONTOKEN&amp;renewer=&lt;USER&gt;"
+    </source>
+The client receives a response with a <a href="#Token"><code>Token</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "Token":
+  {
+    "urlString": "JQAIaG9y..."
+  }
+}
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#renewer"><code>renewer</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/filesystem/getDelegationToken">FileSystem.getDelegationToken</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="RENEWDELEGATIONTOKEN">
+        <title>Renew Delegation Token</title>
+<ul>
+  <li>Submit a HTTP PUT request.
+    <source>
+curl -i -X PUT "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&amp;token=&lt;TOKEN&gt;"
+    </source>
+The client receives a response with a <a href="#long"><code>long</code> JSON object</a>:
+    <source>
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"long": 1320962673997}           //the new expiration time
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#token"><code>token</code></a>,
+  DistributedFileSystem.renewDelegationToken
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="CANCELDELEGATIONTOKEN">
+        <title>Cancel Delegation Token</title>
+<ul>
+  <li>Submit a HTTP PUT request.
+    <source>
+curl -i -X PUT "http://&lt;HOST&gt;:&lt;PORT&gt;/webhdfs/v1/?op=CANCELDELEGATIONTOKEN&amp;token=&lt;TOKEN&gt;"
+    </source>
+The client receives a response with zero content length:
+    <source>
+HTTP/1.1 200 OK
+Content-Length: 0
+    </source>
+  </li>
+</ul>
+<p>
+  See also:
+  <a href="#token"><code>token</code></a>,
+  DistributedFileSystem.cancelDelegationToken
+</p>
+      </section>
+    </section>
+<!-- ***************************************************************************** -->
+<!-- ***************************************************************************** -->
+    <section>
+      <title>Error Responses</title>
+<p>
+  When an operation fails, the server may throw an exception.
+  The JSON schema of error responses is defined in <a href="#RemoteException"><code>RemoteException</code> JSON schema</a>.
+  The table below shows the mapping from exceptions to HTTP response codes.
+</p>
+      <section>
+        <title>HTTP Response Codes</title>
+<table>
+<tr><th>Exceptions</th><th>HTTP Response Codes</th></tr>
+<tr><td><code>IllegalArgumentException     </code></td><td><code>400 Bad Request          </code></td></tr>
+<tr><td><code>UnsupportedOperationException</code></td><td><code>400 Bad Request          </code></td></tr>
+<tr><td><code>SecurityException            </code></td><td><code>401 Unauthorized         </code></td></tr>
+<tr><td><code>IOException                  </code></td><td><code>403 Forbidden            </code></td></tr>
+<tr><td><code>FileNotFoundException        </code></td><td><code>404 Not Found            </code></td></tr>
+<tr><td><code>RumtimeException             </code></td><td><code>500 Internal Server Error</code></td></tr>
+</table>
+<p>
+  Below are examples of exception responses.
+</p>
+<!-- ***************************************************************************** -->
+        <section>
+          <title>Illegal Argument Exception</title>
+<source>
+HTTP/1.1 400 Bad Request
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "RemoteException":
+  {
+    "exception"    : "IllegalArgumentException",
+    "javaClassName": "java.lang.IllegalArgumentException",
+    "message"      : "Invalid value for webhdfs parameter \"permission\": ..."
+  }
+}
+</source>
+        </section>
+<!-- ***************************************************************************** -->
+        <section>
+          <title>Security Exception</title>
+<source>
+HTTP/1.1 401 Unauthorized
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "RemoteException":
+  {
+    "exception"    : "SecurityException",
+    "javaClassName": "java.lang.SecurityException",
+    "message"      : "Failed to obtain user group information: ..."
+  }
+}
+</source>
+        </section>
+<!-- ***************************************************************************** -->
+        <section>
+          <title>Access Control Exception</title>
+<source>
+HTTP/1.1 403 Forbidden
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "RemoteException":
+  {
+    "exception"    : "AccessControlException",
+    "javaClassName": "org.apache.hadoop.security.AccessControlException",
+    "message"      : "Permission denied: ..."
+  }
+}
+</source>
+        </section>
+<!-- ***************************************************************************** -->
+        <section>
+          <title>File Not Found Exception</title>
+<source>
+HTTP/1.1 404 Not Found
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "RemoteException":
+  {
+    "exception"    : "FileNotFoundException",
+    "javaClassName": "java.io.FileNotFoundException",
+    "message"      : "File does not exist: /foo/a.patch"
+  }
+}
+</source>
+        </section>
+      </section>
+    </section>
+<!-- ***************************************************************************** -->
+<!-- ***************************************************************************** -->
+    <section>
+      <title>JSON Schemas</title>
+<p>
+All operations, except for <a href="#OPEN"><code>OPEN</code></a>,
+either return a zero-length response or a JSON response .
+For <a href="#OPEN"><code>OPEN</code></a>, the response is an octet-stream.
+The JSON schemas are shown below.
+See <a href="http://tools.ietf.org/id/draft-zyp-json-schema-03.html">draft-zyp-json-schema-03</a>
+for the syntax definitions of the JSON schemas.
+</p>
+      <section id="boolean">
+        <title>Boolean JSON Schema</title>
+<source>
+{
+  "name"      : "boolean",
+  "properties":
+  {
+    "boolean":
+    {
+      "description": "A boolean value",
+      "type"       : "boolean",
+      "required"   : true
+    }
+  }
+}
+</source>
+<p>
+  See also:
+  <a href="#MKDIRS"><code>MKDIRS</code></a>,
+  <a href="#RENAME"><code>RENAME</code></a>,
+  <a href="#DELETE"><code>DELETE</code></a>,
+  <a href="#SETREPLICATION"><code>SETREPLICATION</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="ContentSummary">
+        <title>ContentSummary JSON Schema</title>
+<source>
+{
+  "name"      : "ContentSummary",
+  "properties":
+  {
+    "ContentSummary":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "directoryCount":
+        {
+          "description": "The number of directories.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "fileCount":
+        {
+          "description": "The number of files.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "length":
+        {
+          "description": "The number of bytes used by the content.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "quota":
+        {
+          "description": "The namespace quota of this directory.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "spaceConsumed":
+        {
+          "description": "The disk space consumed by the content.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "spaceQuota":
+        {
+          "description": "The disk space quota.",
+          "type"       : "integer",
+          "required"   : true
+        }
+      }
+    }
+  }
+}
+</source>
+<p>
+  See also:
+  <a href="#GETCONTENTSUMMARY"><code>GETCONTENTSUMMARY</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="FileChecksum">
+        <title>FileChecksum JSON Schema</title>
+<source>
+{
+  "name"      : "FileChecksum",
+  "properties":
+  {
+    "FileChecksum":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "algorithm":
+        {
+          "description": "The name of the checksum algorithm.",
+          "type"       : "string",
+          "required"   : true
+        },
+        "bytes":
+        {
+          "description": "The byte sequence of the checksum in hexadecimal.",
+          "type"       : "string",
+          "required"   : true
+        },
+        "length":
+        {
+          "description": "The length of the bytes (not the length of the string).",
+          "type"       : "integer",
+          "required"   : true
+        }
+      }
+    }
+  }
+}
+</source>
+<p>
+  See also:
+  <a href="#GETFILECHECKSUM"><code>GETFILECHECKSUM</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="FileStatus">
+        <title>FileStatus JSON Schema</title>
+<source>
+{
+  "name"      : "FileStatus",
+  "properties":
+  {
+    "FileStatus": fileStatusProperties      //See <a href="#fileStatusProperties">FileStatus Properties</a>
+  }
+}
+</source>
+<p>
+  See also:
+  <a href="#GETFILESTATUS"><code>GETFILESTATUS</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/FileStatus">FileStatus</a>
+</p>
+      <section id="fileStatusProperties">
+        <title>FileStatus Properties</title>
+<p>
+  JavaScript syntax is used to define <code>fileStatusProperties</code>
+  so that it can be referred in both <code>FileStatus</code> and <code>FileStatuses</code> JSON schemas.
+</p>
+<source>
+var fileStatusProperties =
+{
+  "type"      : "object",
+  "properties":
+  {
+    "accessTime":
+    {
+      "description": "The access time.",
+      "type"       : "integer",
+      "required"   : true
+    },
+    "blockSize":
+    {
+      "description": "The block size of a file.",
+      "type"       : "integer",
+      "required"   : true
+    },
+    "group":
+    {
+      "description": "The group owner.",
+      "type"       : "string",
+      "required"   : true
+    },
+    "length":
+    {
+      "description": "The number of bytes in a file.",
+      "type"       : "integer",
+      "required"   : true
+    },
+    "modificationTime":
+    {
+      "description": "The modification time.",
+      "type"       : "integer",
+      "required"   : true
+    },
+    "owner":
+    {
+      "description": "The user who is the owner.",
+      "type"       : "string",
+      "required"   : true
+    },
+    "pathSuffix":
+    {
+      "description": "The path suffix.",
+      "type"       : "string",
+      "required"   : true
+    },
+    "permission":
+    {
+      "description": "The permission represented as a octal string.",
+      "type"       : "string",
+      "required"   : true
+    },
+    "replication":
+    {
+      "description": "The number of replication of a file.",
+      "type"       : "integer",
+      "required"   : true
+    },
+   "symlink":                                         //an optional property
+    {
+      "description": "The link target of a symlink.",
+      "type"       : "string"
+    },
+   "type":
+    {
+      "description": "The type of the path object.",
+      "enum"       : ["FILE", "DIRECTORY", "SYMLINK"],
+      "required"   : true
+    }
+  }
+};
+</source>
+        </section>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="FileStatuses">
+        <title>FileStatuses JSON Schema</title>
+<p>
+  A <code>FileStatuses</code> JSON object represents an array of <code>FileStatus</code> JSON objects.
+</p>
+<source>
+{
+  "name"      : "FileStatuses",
+  "properties":
+  {
+    "FileStatuses":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "FileStatus":
+        {
+          "description": "An array of FileStatus",
+          "type"       : "array",
+          "items"      : fileStatusProperties      //See <a href="#fileStatusProperties">FileStatus Properties</a>
+        }
+      }
+    }
+  }
+}
+</source>
+<p>
+  See also:
+  <a href="#LISTSTATUS"><code>LISTSTATUS</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/FileStatus">FileStatus</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="long">
+        <title>Long JSON Schema</title>
+<source>
+{
+  "name"      : "long",
+  "properties":
+  {
+    "long":
+    {
+      "description": "A long integer value",
+      "type"       : "integer",
+      "required"   : true
+    }
+  }
+}
+</source>
+<p>
+  See also:
+  <a href="#RENEWDELEGATIONTOKEN"><code>RENEWDELEGATIONTOKEN</code></a>,
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="Path">
+        <title>Path JSON Schema</title>
+<source>
+{
+  "name"      : "Path",
+  "properties":
+  {
+    "Path":
+    {
+      "description": "The string representation a Path.",
+      "type"       : "string",
+      "required"   : true
+    }
+  }
+}
+</source>
+<p>
+  See also:
+  <a href="#GETHOMEDIRECTORY"><code>GETHOMEDIRECTORY</code></a>,
+  <a href="ext:api/org/apache/hadoop/fs/Path">Path</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="RemoteException">
+        <title>RemoteException JSON Schema</title>
+<source>
+{
+  "name"      : "RemoteException",
+  "properties":
+  {
+    "RemoteException":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "exception":
+        {
+          "description": "Name of the exception",
+          "type"       : "string",
+          "required"   : true
+        },
+        "message":
+        {
+          "description": "Exception message",
+          "type"       : "string",
+          "required"   : true
+        },
+        "javaClassName":                                     //an optional property
+        {
+          "description": "Java class name of the exception",
+          "type"       : "string",
+        }
+      }
+    }
+  }
+}
+</source>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="Token">
+        <title>Token JSON Schema</title>
+<source>
+{
+  "name"      : "Token",
+  "properties":
+  {
+    "Token":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "urlString":
+        {
+          "description": "A delegation token encoded as a URL safe string.",
+          "type"       : "string",
+          "required"   : true
+        }
+      }
+    }
+  }
+}
+</source>
+<p>
+  See also:
+  <a href="#GETDELEGATIONTOKEN"><code>GETDELEGATIONTOKEN</code></a>,
+  the note in <a href="#delegation">Delegation</a>.
+</p>
+      </section>
+    </section>
+<!-- ***************************************************************************** -->
+<!-- ***************************************************************************** -->
+    <section id="ParameterDictionary">
+      <title>HTTP Query Parameter Dictionary</title>
+      <section id="accesstime">
+        <title>Access Time</title>
+<table>
+  <tr><td>Name</td><td><code>accesstime</code></td></tr>
+  <tr><td>Description</td><td>The access time of a file/directory.</td></tr>
+  <tr><td>Type</td><td>long</td></tr>
+  <tr><td>Default Value</td><td>-1 (means keeping it unchanged)</td></tr>
+  <tr><td>Valid Values</td><td>-1 or a timestamp</td></tr>
+  <tr><td>Syntax</td><td>Any integer.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#SETTIMES"><code>SETTIMES</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="blocksize">
+        <title>Block Size</title>
+<table>
+  <tr><td>Name</td><td><code>blocksize</code></td></tr>
+  <tr><td>Description</td><td>The block size of a file.</td></tr>
+  <tr><td>Type</td><td>long</td></tr>
+  <tr><td>Default Value</td><td>Specified in the configuration.</td></tr>
+  <tr><td>Valid Values</td><td>&gt; 0</td></tr>
+  <tr><td>Syntax</td><td>Any integer.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#CREATE"><code>CREATE</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="buffersize">
+        <title>Buffer Size</title>
+<table>
+  <tr><td>Name</td><td><code>buffersize</code></td></tr>
+  <tr><td>Description</td><td>The size of the buffer used in transferring data.</td></tr>
+  <tr><td>Type</td><td>int</td></tr>
+  <tr><td>Default Value</td><td>Specified in the configuration.</td></tr>
+  <tr><td>Valid Values</td><td>&gt; 0</td></tr>
+  <tr><td>Syntax</td><td>Any integer.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#CREATE"><code>CREATE</code></a>,
+  <a href="#APPEND"><code>APPEND</code></a>,
+  <a href="#OPEN"><code>OPEN</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="delegation">
+        <title>Delegation</title>
+<table>
+  <tr><td>Name</td><td><code>delegation</code></td></tr>
+  <tr><td>Description</td><td>The delegation token used for authentication.</td></tr>
+  <tr><td>Type</td><td>String</td></tr>
+  <tr><td>Default Value</td><td>&lt;empty&gt;</td></tr>
+  <tr><td>Valid Values</td><td>An encoded token.</td></tr>
+  <tr><td>Syntax</td><td>See the note below.</td></tr>
+</table>
+<p>
+  <strong>Note</strong> that delegation tokens are encoded as a URL safe string;
+  see <code>encodeToUrlString()</code>
+  and <code>decodeFromUrlString(String)</code>
+  in <code>org.apache.hadoop.security.token.Token</code> for the details of the encoding.
+</p>
+<p>
+  See also:
+  <a href="#Authentication">Authentication</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="destination">
+        <title>Destination</title>
+<table>
+  <tr><td>Name</td><td><code>destination</code></td></tr>
+  <tr><td>Description</td><td>The destination path used in <a href="#RENAME">RENAME</a>.</td></tr>
+  <tr><td>Type</td><td>Path</td></tr>
+  <tr><td>Default Value</td><td>&lt;empty&gt; (an invalid path)</td></tr>
+  <tr><td>Valid Values</td><td>An absolute FileSystem path without scheme and authority.</td></tr>
+  <tr><td>Syntax</td><td>Any path.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#RENAME"><code>RENAME</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="doas">
+        <title>Do As</title>
+<table>
+  <tr><td>Name</td><td><code>doas</code></td></tr>
+  <tr><td>Description</td><td>Allowing a proxy user to do as another user.</td></tr>
+  <tr><td>Type</td><td>String</td></tr>
+  <tr><td>Default Value</td><td>null</td></tr>
+  <tr><td>Valid Values</td><td>Any valid username.</td></tr>
+  <tr><td>Syntax</td><td>Any string.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#ProxyUsers">Proxy Users</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="group">
+        <title>Group</title>
+<table>
+  <tr><td>Name</td><td><code>group</code></td></tr>
+  <tr><td>Description</td><td>The name of a group.</td></tr>
+  <tr><td>Type</td><td>String</td></tr>
+  <tr><td>Default Value</td><td>&lt;empty&gt; (means keeping it unchanged)</td></tr>
+  <tr><td>Valid Values</td><td>Any valid group name.</td></tr>
+  <tr><td>Syntax</td><td>Any string.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#SETOWNER"><code>SETOWNER</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="length">
+        <title>Length</title>
+<table>
+  <tr><td>Name</td><td><code>length</code></td></tr>
+  <tr><td>Description</td><td>The number of bytes to be processed.</td></tr>
+  <tr><td>Type</td><td>long</td></tr>
+  <tr><td>Default Value</td><td>null (means the entire file)</td></tr>
+  <tr><td>Valid Values</td><td>&gt;= 0 or null</td></tr>
+  <tr><td>Syntax</td><td>Any integer.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#OPEN"><code>OPEN</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="modificationtime">
+        <title>Modification Time</title>
+<table>
+  <tr><td>Name</td><td><code>modificationtime</code></td></tr>
+  <tr><td>Description</td><td>The modification time of a file/directory.</td></tr>
+  <tr><td>Type</td><td>long</td></tr>
+  <tr><td>Default Value</td><td>-1 (means keeping it unchanged)</td></tr>
+  <tr><td>Valid Values</td><td>-1 or a timestamp</td></tr>
+  <tr><td>Syntax</td><td>Any integer.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#SETTIMES"><code>SETTIMES</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="offset">
+        <title>Offset</title>
+<table>
+  <tr><td>Name</td><td><code>offset</code></td></tr>
+  <tr><td>Description</td><td>The starting byte position.</td></tr>
+  <tr><td>Type</td><td>long</td></tr>
+  <tr><td>Default Value</td><td>0</td></tr>
+  <tr><td>Valid Values</td><td>&gt;= 0</td></tr>
+  <tr><td>Syntax</td><td>Any integer.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#OPEN"><code>OPEN</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="op">
+        <title>Op</title>
+<table>
+  <tr><td>Name</td><td><code>op</code></td></tr>
+  <tr><td>Description</td><td>The name of the operation to be executed.</td></tr>
+  <tr><td>Type</td><td>enum</td></tr>
+  <tr><td>Default Value</td><td>null (an invalid value)</td></tr>
+  <tr><td>Valid Values</td><td>Any valid operation name.</td></tr>
+  <tr><td>Syntax</td><td>Any string.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#Operations">Operations</a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="overwrite">
+        <title>Overwrite</title>
+<table>
+  <tr><td>Name</td><td><code>overwrite</code></td></tr>
+  <tr><td>Description</td><td>If a file already exists, should it be overwritten?</td></tr>
+  <tr><td>Type</td><td>boolean</td></tr>
+  <tr><td>Default Value</td><td>false</td></tr>
+  <tr><td>Valid Values</td><td>true | false</td></tr>
+  <tr><td>Syntax</td><td>true | false</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#CREATE"><code>CREATE</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="owner">
+        <title>Owner</title>
+<table>
+  <tr><td>Name</td><td><code>owner</code></td></tr>
+  <tr><td>Description</td><td>The username who is the owner of a file/directory.</td></tr>
+  <tr><td>Type</td><td>String</td></tr>
+  <tr><td>Default Value</td><td>&lt;empty&gt; (means keeping it unchanged)</td></tr>
+  <tr><td>Valid Values</td><td>Any valid username.</td></tr>
+  <tr><td>Syntax</td><td>Any string.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#SETOWNER"><code>SETOWNER</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="permission">
+        <title>Permission</title>
+<table>
+  <tr><td>Name</td><td><code>permission</code></td></tr>
+  <tr><td>Description</td><td>The permission of a file/directory.</td></tr>
+  <tr><td>Type</td><td>Octal</td></tr>
+  <tr><td>Default Value</td><td>755</td></tr>
+  <tr><td>Valid Values</td><td>0 - 1777</td></tr>
+  <tr><td>Syntax</td><td>Any radix-8 integer (leading zeros may be omitted.)</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#CREATE"><code>CREATE</code></a>,
+  <a href="#MKDIRS"><code>MKDIRS</code></a>,
+  <a href="#SETPERMISSION"><code>SETPERMISSION</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="recursive">
+        <title>Recursive</title>
+<table>
+  <tr><td>Name</td><td><code>recursive</code></td></tr>
+  <tr><td>Description</td><td>Should the operation act on the content in the subdirectories?</td></tr>
+  <tr><td>Type</td><td>boolean</td></tr>
+  <tr><td>Default Value</td><td>false</td></tr>
+  <tr><td>Valid Values</td><td>true | false</td></tr>
+  <tr><td>Syntax</td><td>true | false</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#RENAME"><code>RENAME</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="renewer">
+        <title>Renewer</title>
+<table>
+  <tr><td>Name</td><td><code>renewer</code></td></tr>
+  <tr><td>Description</td><td>The username of the renewer of a delegation token.</td></tr>
+  <tr><td>Type</td><td>String</td></tr>
+  <tr><td>Default Value</td><td>&lt;empty&gt; (means the current user)</td></tr>
+  <tr><td>Valid Values</td><td>Any valid username.</td></tr>
+  <tr><td>Syntax</td><td>Any string.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#GETDELEGATIONTOKEN"><code>GETDELEGATIONTOKEN</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="replication">
+        <title>Replication</title>
+<table>
+  <tr><td>Name</td><td><code>replication</code></td></tr>
+  <tr><td>Description</td><td>The number of replications of a file.</td></tr>
+  <tr><td>Type</td><td>short</td></tr>
+  <tr><td>Default Value</td><td>Specified in the configuration.</td></tr>
+  <tr><td>Valid Values</td><td>&gt; 0</td></tr>
+  <tr><td>Syntax</td><td>Any integer.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#CREATE"><code>CREATE</code></a>,
+  <a href="#SETREPLICATION"><code>SETREPLICATION</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="token">
+        <title>Token</title>
+<table>
+  <tr><td>Name</td><td><code>token</code></td></tr>
+  <tr><td>Description</td><td>The delegation token used for the operation.</td></tr>
+  <tr><td>Type</td><td>String</td></tr>
+  <tr><td>Default Value</td><td>&lt;empty&gt;</td></tr>
+  <tr><td>Valid Values</td><td>An encoded token.</td></tr>
+  <tr><td>Syntax</td><td>See the note in <a href="#delegation">Delegation</a>.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#RENEWDELEGATIONTOKEN"><code>RENEWDELEGATIONTOKEN</code></a>,
+  <a href="#CANCELDELEGATIONTOKEN"><code>CANCELDELEGATIONTOKEN</code></a>
+</p>
+      </section>
+<!-- ***************************************************************************** -->
+      <section id="user.name">
+        <title>Username</title>
+<table>
+  <tr><td>Name</td><td><code>user.name</code></td></tr>
+  <tr><td>Description</td><td>The authenticated user; see <a href="#Authentication">Authentication</a>.</td></tr>
+  <tr><td>Type</td><td>String</td></tr>
+  <tr><td>Default Value</td><td>null</td></tr>
+  <tr><td>Valid Values</td><td>Any valid username.</td></tr>
+  <tr><td>Syntax</td><td>Any string.</td></tr>
+</table>
+<p>
+  See also:
+  <a href="#Authentication">Authentication</a>
+</p>
+      </section>
+    </section>
+  </body>
+</document>

+ 17153 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/DatanodeProtocolProtos.java

@@ -0,0 +1,17153 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: DatanodeProtocol.proto
+
+package org.apache.hadoop.hdfs.protocol.proto;
+
+public final class DatanodeProtocolProtos {
+  private DatanodeProtocolProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface DatanodeRegistrationProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeIDProto datanodeID = 1;
+    boolean hasDatanodeID();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDatanodeID();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getDatanodeIDOrBuilder();
+    
+    // required .StorageInfoProto storateInfo = 2;
+    boolean hasStorateInfo();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto getStorateInfo();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProtoOrBuilder getStorateInfoOrBuilder();
+    
+    // required .ExportedBlockKeysProto keys = 3;
+    boolean hasKeys();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto getKeys();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder getKeysOrBuilder();
+  }
+  public static final class DatanodeRegistrationProto extends
+      com.google.protobuf.GeneratedMessage
+      implements DatanodeRegistrationProtoOrBuilder {
+    // Use DatanodeRegistrationProto.newBuilder() to construct.
+    private DatanodeRegistrationProto(Builder builder) {
+      super(builder);
+    }
+    private DatanodeRegistrationProto(boolean noInit) {}
+    
+    private static final DatanodeRegistrationProto defaultInstance;
+    public static DatanodeRegistrationProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public DatanodeRegistrationProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_DatanodeRegistrationProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_DatanodeRegistrationProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .DatanodeIDProto datanodeID = 1;
+    public static final int DATANODEID_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto datanodeID_;
+    public boolean hasDatanodeID() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDatanodeID() {
+      return datanodeID_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getDatanodeIDOrBuilder() {
+      return datanodeID_;
+    }
+    
+    // required .StorageInfoProto storateInfo = 2;
+    public static final int STORATEINFO_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto storateInfo_;
+    public boolean hasStorateInfo() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto getStorateInfo() {
+      return storateInfo_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProtoOrBuilder getStorateInfoOrBuilder() {
+      return storateInfo_;
+    }
+    
+    // required .ExportedBlockKeysProto keys = 3;
+    public static final int KEYS_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto keys_;
+    public boolean hasKeys() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto getKeys() {
+      return keys_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder getKeysOrBuilder() {
+      return keys_;
+    }
+    
+    private void initFields() {
+      datanodeID_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+      storateInfo_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.getDefaultInstance();
+      keys_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasDatanodeID()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasStorateInfo()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasKeys()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getDatanodeID().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getStorateInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getKeys().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, datanodeID_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, storateInfo_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, keys_);
+      }
+      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, datanodeID_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, storateInfo_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, keys_);
+      }
+      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.DatanodeProtocolProtos.DatanodeRegistrationProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto) obj;
+      
+      boolean result = true;
+      result = result && (hasDatanodeID() == other.hasDatanodeID());
+      if (hasDatanodeID()) {
+        result = result && getDatanodeID()
+            .equals(other.getDatanodeID());
+      }
+      result = result && (hasStorateInfo() == other.hasStorateInfo());
+      if (hasStorateInfo()) {
+        result = result && getStorateInfo()
+            .equals(other.getStorateInfo());
+      }
+      result = result && (hasKeys() == other.hasKeys());
+      if (hasKeys()) {
+        result = result && getKeys()
+            .equals(other.getKeys());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasDatanodeID()) {
+        hash = (37 * hash) + DATANODEID_FIELD_NUMBER;
+        hash = (53 * hash) + getDatanodeID().hashCode();
+      }
+      if (hasStorateInfo()) {
+        hash = (37 * hash) + STORATEINFO_FIELD_NUMBER;
+        hash = (53 * hash) + getStorateInfo().hashCode();
+      }
+      if (hasKeys()) {
+        hash = (37 * hash) + KEYS_FIELD_NUMBER;
+        hash = (53 * hash) + getKeys().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto 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.DatanodeProtocolProtos.DatanodeRegistrationProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto 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.DatanodeProtocolProtos.DatanodeRegistrationProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto 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.DatanodeProtocolProtos.DatanodeRegistrationProto 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.DatanodeProtocolProtos.DatanodeRegistrationProto 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.DatanodeProtocolProtos.DatanodeRegistrationProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto 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.DatanodeProtocolProtos.DatanodeRegistrationProto 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.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_DatanodeRegistrationProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_DatanodeRegistrationProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getDatanodeIDFieldBuilder();
+          getStorateInfoFieldBuilder();
+          getKeysFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (datanodeIDBuilder_ == null) {
+          datanodeID_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+        } else {
+          datanodeIDBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (storateInfoBuilder_ == null) {
+          storateInfo_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.getDefaultInstance();
+        } else {
+          storateInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (keysBuilder_ == null) {
+          keys_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance();
+        } else {
+          keysBuilder_.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.DatanodeProtocolProtos.DatanodeRegistrationProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (datanodeIDBuilder_ == null) {
+          result.datanodeID_ = datanodeID_;
+        } else {
+          result.datanodeID_ = datanodeIDBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (storateInfoBuilder_ == null) {
+          result.storateInfo_ = storateInfo_;
+        } else {
+          result.storateInfo_ = storateInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (keysBuilder_ == null) {
+          result.keys_ = keys_;
+        } else {
+          result.keys_ = keysBuilder_.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.DatanodeProtocolProtos.DatanodeRegistrationProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance()) return this;
+        if (other.hasDatanodeID()) {
+          mergeDatanodeID(other.getDatanodeID());
+        }
+        if (other.hasStorateInfo()) {
+          mergeStorateInfo(other.getStorateInfo());
+        }
+        if (other.hasKeys()) {
+          mergeKeys(other.getKeys());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasDatanodeID()) {
+          
+          return false;
+        }
+        if (!hasStorateInfo()) {
+          
+          return false;
+        }
+        if (!hasKeys()) {
+          
+          return false;
+        }
+        if (!getDatanodeID().isInitialized()) {
+          
+          return false;
+        }
+        if (!getStorateInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (!getKeys().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 (hasDatanodeID()) {
+                subBuilder.mergeFrom(getDatanodeID());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setDatanodeID(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.newBuilder();
+              if (hasStorateInfo()) {
+                subBuilder.mergeFrom(getStorateInfo());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setStorateInfo(subBuilder.buildPartial());
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.newBuilder();
+              if (hasKeys()) {
+                subBuilder.mergeFrom(getKeys());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setKeys(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeIDProto datanodeID = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto datanodeID_ = 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> datanodeIDBuilder_;
+      public boolean hasDatanodeID() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDatanodeID() {
+        if (datanodeIDBuilder_ == null) {
+          return datanodeID_;
+        } else {
+          return datanodeIDBuilder_.getMessage();
+        }
+      }
+      public Builder setDatanodeID(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
+        if (datanodeIDBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          datanodeID_ = value;
+          onChanged();
+        } else {
+          datanodeIDBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setDatanodeID(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder builderForValue) {
+        if (datanodeIDBuilder_ == null) {
+          datanodeID_ = builderForValue.build();
+          onChanged();
+        } else {
+          datanodeIDBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeDatanodeID(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
+        if (datanodeIDBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              datanodeID_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance()) {
+            datanodeID_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.newBuilder(datanodeID_).mergeFrom(value).buildPartial();
+          } else {
+            datanodeID_ = value;
+          }
+          onChanged();
+        } else {
+          datanodeIDBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearDatanodeID() {
+        if (datanodeIDBuilder_ == null) {
+          datanodeID_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+          onChanged();
+        } else {
+          datanodeIDBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder getDatanodeIDBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getDatanodeIDFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getDatanodeIDOrBuilder() {
+        if (datanodeIDBuilder_ != null) {
+          return datanodeIDBuilder_.getMessageOrBuilder();
+        } else {
+          return datanodeID_;
+        }
+      }
+      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> 
+          getDatanodeIDFieldBuilder() {
+        if (datanodeIDBuilder_ == null) {
+          datanodeIDBuilder_ = 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>(
+                  datanodeID_,
+                  getParentForChildren(),
+                  isClean());
+          datanodeID_ = null;
+        }
+        return datanodeIDBuilder_;
+      }
+      
+      // required .StorageInfoProto storateInfo = 2;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto storateInfo_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProtoOrBuilder> storateInfoBuilder_;
+      public boolean hasStorateInfo() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto getStorateInfo() {
+        if (storateInfoBuilder_ == null) {
+          return storateInfo_;
+        } else {
+          return storateInfoBuilder_.getMessage();
+        }
+      }
+      public Builder setStorateInfo(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto value) {
+        if (storateInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          storateInfo_ = value;
+          onChanged();
+        } else {
+          storateInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder setStorateInfo(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.Builder builderForValue) {
+        if (storateInfoBuilder_ == null) {
+          storateInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          storateInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder mergeStorateInfo(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto value) {
+        if (storateInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              storateInfo_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.getDefaultInstance()) {
+            storateInfo_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.newBuilder(storateInfo_).mergeFrom(value).buildPartial();
+          } else {
+            storateInfo_ = value;
+          }
+          onChanged();
+        } else {
+          storateInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder clearStorateInfo() {
+        if (storateInfoBuilder_ == null) {
+          storateInfo_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.getDefaultInstance();
+          onChanged();
+        } else {
+          storateInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.Builder getStorateInfoBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getStorateInfoFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProtoOrBuilder getStorateInfoOrBuilder() {
+        if (storateInfoBuilder_ != null) {
+          return storateInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return storateInfo_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProtoOrBuilder> 
+          getStorateInfoFieldBuilder() {
+        if (storateInfoBuilder_ == null) {
+          storateInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProtoOrBuilder>(
+                  storateInfo_,
+                  getParentForChildren(),
+                  isClean());
+          storateInfo_ = null;
+        }
+        return storateInfoBuilder_;
+      }
+      
+      // required .ExportedBlockKeysProto keys = 3;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto keys_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder> keysBuilder_;
+      public boolean hasKeys() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto getKeys() {
+        if (keysBuilder_ == null) {
+          return keys_;
+        } else {
+          return keysBuilder_.getMessage();
+        }
+      }
+      public Builder setKeys(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto value) {
+        if (keysBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          keys_ = value;
+          onChanged();
+        } else {
+          keysBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder setKeys(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder builderForValue) {
+        if (keysBuilder_ == null) {
+          keys_ = builderForValue.build();
+          onChanged();
+        } else {
+          keysBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder mergeKeys(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto value) {
+        if (keysBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              keys_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance()) {
+            keys_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.newBuilder(keys_).mergeFrom(value).buildPartial();
+          } else {
+            keys_ = value;
+          }
+          onChanged();
+        } else {
+          keysBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder clearKeys() {
+        if (keysBuilder_ == null) {
+          keys_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance();
+          onChanged();
+        } else {
+          keysBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder getKeysBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getKeysFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder getKeysOrBuilder() {
+        if (keysBuilder_ != null) {
+          return keysBuilder_.getMessageOrBuilder();
+        } else {
+          return keys_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder> 
+          getKeysFieldBuilder() {
+        if (keysBuilder_ == null) {
+          keysBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder>(
+                  keys_,
+                  getParentForChildren(),
+                  isClean());
+          keys_ = null;
+        }
+        return keysBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:DatanodeRegistrationProto)
+    }
+    
+    static {
+      defaultInstance = new DatanodeRegistrationProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:DatanodeRegistrationProto)
+  }
+  
+  public interface DatanodeCommandProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeCommandProto.Type cmdType = 1;
+    boolean hasCmdType();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type getCmdType();
+    
+    // optional .BalancerBandwidthCommandProto balancerCmd = 2;
+    boolean hasBalancerCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto getBalancerCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProtoOrBuilder getBalancerCmdOrBuilder();
+    
+    // optional .BlockCommandProto blkCmd = 3;
+    boolean hasBlkCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto getBlkCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProtoOrBuilder getBlkCmdOrBuilder();
+    
+    // optional .BlockRecoveryCommndProto recoveryCmd = 4;
+    boolean hasRecoveryCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto getRecoveryCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProtoOrBuilder getRecoveryCmdOrBuilder();
+    
+    // optional .FinalizeCommandProto finalizeCmd = 5;
+    boolean hasFinalizeCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto getFinalizeCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProtoOrBuilder getFinalizeCmdOrBuilder();
+    
+    // optional .KeyUpdateCommandProto keyUpdateCmd = 6;
+    boolean hasKeyUpdateCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto getKeyUpdateCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProtoOrBuilder getKeyUpdateCmdOrBuilder();
+    
+    // optional .RegisterCommandProto registerCmd = 7;
+    boolean hasRegisterCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto getRegisterCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProtoOrBuilder getRegisterCmdOrBuilder();
+    
+    // optional .UpgradeCommandProto upgradeCmd = 8;
+    boolean hasUpgradeCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getUpgradeCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder getUpgradeCmdOrBuilder();
+  }
+  public static final class DatanodeCommandProto extends
+      com.google.protobuf.GeneratedMessage
+      implements DatanodeCommandProtoOrBuilder {
+    // Use DatanodeCommandProto.newBuilder() to construct.
+    private DatanodeCommandProto(Builder builder) {
+      super(builder);
+    }
+    private DatanodeCommandProto(boolean noInit) {}
+    
+    private static final DatanodeCommandProto defaultInstance;
+    public static DatanodeCommandProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public DatanodeCommandProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_DatanodeCommandProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_DatanodeCommandProto_fieldAccessorTable;
+    }
+    
+    public enum Type
+        implements com.google.protobuf.ProtocolMessageEnum {
+      BalancerBandwidthCommand(0, 0),
+      BlockCommand(1, 1),
+      BlockRecoveryCommand(2, 2),
+      FinalizeCommand(3, 3),
+      KeyUpdateCommand(4, 4),
+      RegisterCommand(5, 5),
+      UpgradeCommand(6, 6),
+      ;
+      
+      public static final int BalancerBandwidthCommand_VALUE = 0;
+      public static final int BlockCommand_VALUE = 1;
+      public static final int BlockRecoveryCommand_VALUE = 2;
+      public static final int FinalizeCommand_VALUE = 3;
+      public static final int KeyUpdateCommand_VALUE = 4;
+      public static final int RegisterCommand_VALUE = 5;
+      public static final int UpgradeCommand_VALUE = 6;
+      
+      
+      public final int getNumber() { return value; }
+      
+      public static Type valueOf(int value) {
+        switch (value) {
+          case 0: return BalancerBandwidthCommand;
+          case 1: return BlockCommand;
+          case 2: return BlockRecoveryCommand;
+          case 3: return FinalizeCommand;
+          case 4: return KeyUpdateCommand;
+          case 5: return RegisterCommand;
+          case 6: return UpgradeCommand;
+          default: return null;
+        }
+      }
+      
+      public static com.google.protobuf.Internal.EnumLiteMap<Type>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<Type>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<Type>() {
+              public Type findValueByNumber(int number) {
+                return Type.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.DatanodeProtocolProtos.DatanodeCommandProto.getDescriptor().getEnumTypes().get(0);
+      }
+      
+      private static final Type[] VALUES = {
+        BalancerBandwidthCommand, BlockCommand, BlockRecoveryCommand, FinalizeCommand, KeyUpdateCommand, RegisterCommand, UpgradeCommand, 
+      };
+      
+      public static Type 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 Type(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+      
+      // @@protoc_insertion_point(enum_scope:DatanodeCommandProto.Type)
+    }
+    
+    private int bitField0_;
+    // required .DatanodeCommandProto.Type cmdType = 1;
+    public static final int CMDTYPE_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type cmdType_;
+    public boolean hasCmdType() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type getCmdType() {
+      return cmdType_;
+    }
+    
+    // optional .BalancerBandwidthCommandProto balancerCmd = 2;
+    public static final int BALANCERCMD_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto balancerCmd_;
+    public boolean hasBalancerCmd() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto getBalancerCmd() {
+      return balancerCmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProtoOrBuilder getBalancerCmdOrBuilder() {
+      return balancerCmd_;
+    }
+    
+    // optional .BlockCommandProto blkCmd = 3;
+    public static final int BLKCMD_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto blkCmd_;
+    public boolean hasBlkCmd() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto getBlkCmd() {
+      return blkCmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProtoOrBuilder getBlkCmdOrBuilder() {
+      return blkCmd_;
+    }
+    
+    // optional .BlockRecoveryCommndProto recoveryCmd = 4;
+    public static final int RECOVERYCMD_FIELD_NUMBER = 4;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto recoveryCmd_;
+    public boolean hasRecoveryCmd() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto getRecoveryCmd() {
+      return recoveryCmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProtoOrBuilder getRecoveryCmdOrBuilder() {
+      return recoveryCmd_;
+    }
+    
+    // optional .FinalizeCommandProto finalizeCmd = 5;
+    public static final int FINALIZECMD_FIELD_NUMBER = 5;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto finalizeCmd_;
+    public boolean hasFinalizeCmd() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto getFinalizeCmd() {
+      return finalizeCmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProtoOrBuilder getFinalizeCmdOrBuilder() {
+      return finalizeCmd_;
+    }
+    
+    // optional .KeyUpdateCommandProto keyUpdateCmd = 6;
+    public static final int KEYUPDATECMD_FIELD_NUMBER = 6;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto keyUpdateCmd_;
+    public boolean hasKeyUpdateCmd() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto getKeyUpdateCmd() {
+      return keyUpdateCmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProtoOrBuilder getKeyUpdateCmdOrBuilder() {
+      return keyUpdateCmd_;
+    }
+    
+    // optional .RegisterCommandProto registerCmd = 7;
+    public static final int REGISTERCMD_FIELD_NUMBER = 7;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto registerCmd_;
+    public boolean hasRegisterCmd() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto getRegisterCmd() {
+      return registerCmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProtoOrBuilder getRegisterCmdOrBuilder() {
+      return registerCmd_;
+    }
+    
+    // optional .UpgradeCommandProto upgradeCmd = 8;
+    public static final int UPGRADECMD_FIELD_NUMBER = 8;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto upgradeCmd_;
+    public boolean hasUpgradeCmd() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getUpgradeCmd() {
+      return upgradeCmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder getUpgradeCmdOrBuilder() {
+      return upgradeCmd_;
+    }
+    
+    private void initFields() {
+      cmdType_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type.BalancerBandwidthCommand;
+      balancerCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.getDefaultInstance();
+      blkCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.getDefaultInstance();
+      recoveryCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.getDefaultInstance();
+      finalizeCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.getDefaultInstance();
+      keyUpdateCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.getDefaultInstance();
+      registerCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.getDefaultInstance();
+      upgradeCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasCmdType()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (hasBalancerCmd()) {
+        if (!getBalancerCmd().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      if (hasBlkCmd()) {
+        if (!getBlkCmd().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      if (hasRecoveryCmd()) {
+        if (!getRecoveryCmd().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      if (hasFinalizeCmd()) {
+        if (!getFinalizeCmd().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      if (hasKeyUpdateCmd()) {
+        if (!getKeyUpdateCmd().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      if (hasUpgradeCmd()) {
+        if (!getUpgradeCmd().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, cmdType_.getNumber());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, balancerCmd_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, blkCmd_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeMessage(4, recoveryCmd_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeMessage(5, finalizeCmd_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeMessage(6, keyUpdateCmd_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeMessage(7, registerCmd_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeMessage(8, upgradeCmd_);
+      }
+      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, cmdType_.getNumber());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, balancerCmd_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, blkCmd_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, recoveryCmd_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, finalizeCmd_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(6, keyUpdateCmd_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(7, registerCmd_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(8, upgradeCmd_);
+      }
+      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.DatanodeProtocolProtos.DatanodeCommandProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto) obj;
+      
+      boolean result = true;
+      result = result && (hasCmdType() == other.hasCmdType());
+      if (hasCmdType()) {
+        result = result &&
+            (getCmdType() == other.getCmdType());
+      }
+      result = result && (hasBalancerCmd() == other.hasBalancerCmd());
+      if (hasBalancerCmd()) {
+        result = result && getBalancerCmd()
+            .equals(other.getBalancerCmd());
+      }
+      result = result && (hasBlkCmd() == other.hasBlkCmd());
+      if (hasBlkCmd()) {
+        result = result && getBlkCmd()
+            .equals(other.getBlkCmd());
+      }
+      result = result && (hasRecoveryCmd() == other.hasRecoveryCmd());
+      if (hasRecoveryCmd()) {
+        result = result && getRecoveryCmd()
+            .equals(other.getRecoveryCmd());
+      }
+      result = result && (hasFinalizeCmd() == other.hasFinalizeCmd());
+      if (hasFinalizeCmd()) {
+        result = result && getFinalizeCmd()
+            .equals(other.getFinalizeCmd());
+      }
+      result = result && (hasKeyUpdateCmd() == other.hasKeyUpdateCmd());
+      if (hasKeyUpdateCmd()) {
+        result = result && getKeyUpdateCmd()
+            .equals(other.getKeyUpdateCmd());
+      }
+      result = result && (hasRegisterCmd() == other.hasRegisterCmd());
+      if (hasRegisterCmd()) {
+        result = result && getRegisterCmd()
+            .equals(other.getRegisterCmd());
+      }
+      result = result && (hasUpgradeCmd() == other.hasUpgradeCmd());
+      if (hasUpgradeCmd()) {
+        result = result && getUpgradeCmd()
+            .equals(other.getUpgradeCmd());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasCmdType()) {
+        hash = (37 * hash) + CMDTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getCmdType());
+      }
+      if (hasBalancerCmd()) {
+        hash = (37 * hash) + BALANCERCMD_FIELD_NUMBER;
+        hash = (53 * hash) + getBalancerCmd().hashCode();
+      }
+      if (hasBlkCmd()) {
+        hash = (37 * hash) + BLKCMD_FIELD_NUMBER;
+        hash = (53 * hash) + getBlkCmd().hashCode();
+      }
+      if (hasRecoveryCmd()) {
+        hash = (37 * hash) + RECOVERYCMD_FIELD_NUMBER;
+        hash = (53 * hash) + getRecoveryCmd().hashCode();
+      }
+      if (hasFinalizeCmd()) {
+        hash = (37 * hash) + FINALIZECMD_FIELD_NUMBER;
+        hash = (53 * hash) + getFinalizeCmd().hashCode();
+      }
+      if (hasKeyUpdateCmd()) {
+        hash = (37 * hash) + KEYUPDATECMD_FIELD_NUMBER;
+        hash = (53 * hash) + getKeyUpdateCmd().hashCode();
+      }
+      if (hasRegisterCmd()) {
+        hash = (37 * hash) + REGISTERCMD_FIELD_NUMBER;
+        hash = (53 * hash) + getRegisterCmd().hashCode();
+      }
+      if (hasUpgradeCmd()) {
+        hash = (37 * hash) + UPGRADECMD_FIELD_NUMBER;
+        hash = (53 * hash) + getUpgradeCmd().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto 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.DatanodeProtocolProtos.DatanodeCommandProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto 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.DatanodeProtocolProtos.DatanodeCommandProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto 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.DatanodeProtocolProtos.DatanodeCommandProto 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.DatanodeProtocolProtos.DatanodeCommandProto 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.DatanodeProtocolProtos.DatanodeCommandProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto 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.DatanodeProtocolProtos.DatanodeCommandProto 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.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_DatanodeCommandProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_DatanodeCommandProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBalancerCmdFieldBuilder();
+          getBlkCmdFieldBuilder();
+          getRecoveryCmdFieldBuilder();
+          getFinalizeCmdFieldBuilder();
+          getKeyUpdateCmdFieldBuilder();
+          getRegisterCmdFieldBuilder();
+          getUpgradeCmdFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        cmdType_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type.BalancerBandwidthCommand;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (balancerCmdBuilder_ == null) {
+          balancerCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.getDefaultInstance();
+        } else {
+          balancerCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (blkCmdBuilder_ == null) {
+          blkCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.getDefaultInstance();
+        } else {
+          blkCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        if (recoveryCmdBuilder_ == null) {
+          recoveryCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.getDefaultInstance();
+        } else {
+          recoveryCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        if (finalizeCmdBuilder_ == null) {
+          finalizeCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.getDefaultInstance();
+        } else {
+          finalizeCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000010);
+        if (keyUpdateCmdBuilder_ == null) {
+          keyUpdateCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.getDefaultInstance();
+        } else {
+          keyUpdateCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000020);
+        if (registerCmdBuilder_ == null) {
+          registerCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.getDefaultInstance();
+        } else {
+          registerCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000040);
+        if (upgradeCmdBuilder_ == null) {
+          upgradeCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+        } else {
+          upgradeCmdBuilder_.clear();
+        }
+        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.DatanodeProtocolProtos.DatanodeCommandProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.cmdType_ = cmdType_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (balancerCmdBuilder_ == null) {
+          result.balancerCmd_ = balancerCmd_;
+        } else {
+          result.balancerCmd_ = balancerCmdBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (blkCmdBuilder_ == null) {
+          result.blkCmd_ = blkCmd_;
+        } else {
+          result.blkCmd_ = blkCmdBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        if (recoveryCmdBuilder_ == null) {
+          result.recoveryCmd_ = recoveryCmd_;
+        } else {
+          result.recoveryCmd_ = recoveryCmdBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        if (finalizeCmdBuilder_ == null) {
+          result.finalizeCmd_ = finalizeCmd_;
+        } else {
+          result.finalizeCmd_ = finalizeCmdBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        if (keyUpdateCmdBuilder_ == null) {
+          result.keyUpdateCmd_ = keyUpdateCmd_;
+        } else {
+          result.keyUpdateCmd_ = keyUpdateCmdBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        if (registerCmdBuilder_ == null) {
+          result.registerCmd_ = registerCmd_;
+        } else {
+          result.registerCmd_ = registerCmdBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        if (upgradeCmdBuilder_ == null) {
+          result.upgradeCmd_ = upgradeCmd_;
+        } else {
+          result.upgradeCmd_ = upgradeCmdBuilder_.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.DatanodeProtocolProtos.DatanodeCommandProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.getDefaultInstance()) return this;
+        if (other.hasCmdType()) {
+          setCmdType(other.getCmdType());
+        }
+        if (other.hasBalancerCmd()) {
+          mergeBalancerCmd(other.getBalancerCmd());
+        }
+        if (other.hasBlkCmd()) {
+          mergeBlkCmd(other.getBlkCmd());
+        }
+        if (other.hasRecoveryCmd()) {
+          mergeRecoveryCmd(other.getRecoveryCmd());
+        }
+        if (other.hasFinalizeCmd()) {
+          mergeFinalizeCmd(other.getFinalizeCmd());
+        }
+        if (other.hasKeyUpdateCmd()) {
+          mergeKeyUpdateCmd(other.getKeyUpdateCmd());
+        }
+        if (other.hasRegisterCmd()) {
+          mergeRegisterCmd(other.getRegisterCmd());
+        }
+        if (other.hasUpgradeCmd()) {
+          mergeUpgradeCmd(other.getUpgradeCmd());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasCmdType()) {
+          
+          return false;
+        }
+        if (hasBalancerCmd()) {
+          if (!getBalancerCmd().isInitialized()) {
+            
+            return false;
+          }
+        }
+        if (hasBlkCmd()) {
+          if (!getBlkCmd().isInitialized()) {
+            
+            return false;
+          }
+        }
+        if (hasRecoveryCmd()) {
+          if (!getRecoveryCmd().isInitialized()) {
+            
+            return false;
+          }
+        }
+        if (hasFinalizeCmd()) {
+          if (!getFinalizeCmd().isInitialized()) {
+            
+            return false;
+          }
+        }
+        if (hasKeyUpdateCmd()) {
+          if (!getKeyUpdateCmd().isInitialized()) {
+            
+            return false;
+          }
+        }
+        if (hasUpgradeCmd()) {
+          if (!getUpgradeCmd().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.DatanodeProtocolProtos.DatanodeCommandProto.Type value = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                cmdType_ = value;
+              }
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.newBuilder();
+              if (hasBalancerCmd()) {
+                subBuilder.mergeFrom(getBalancerCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setBalancerCmd(subBuilder.buildPartial());
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.newBuilder();
+              if (hasBlkCmd()) {
+                subBuilder.mergeFrom(getBlkCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setBlkCmd(subBuilder.buildPartial());
+              break;
+            }
+            case 34: {
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.newBuilder();
+              if (hasRecoveryCmd()) {
+                subBuilder.mergeFrom(getRecoveryCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRecoveryCmd(subBuilder.buildPartial());
+              break;
+            }
+            case 42: {
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.newBuilder();
+              if (hasFinalizeCmd()) {
+                subBuilder.mergeFrom(getFinalizeCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setFinalizeCmd(subBuilder.buildPartial());
+              break;
+            }
+            case 50: {
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.newBuilder();
+              if (hasKeyUpdateCmd()) {
+                subBuilder.mergeFrom(getKeyUpdateCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setKeyUpdateCmd(subBuilder.buildPartial());
+              break;
+            }
+            case 58: {
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.newBuilder();
+              if (hasRegisterCmd()) {
+                subBuilder.mergeFrom(getRegisterCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegisterCmd(subBuilder.buildPartial());
+              break;
+            }
+            case 66: {
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.newBuilder();
+              if (hasUpgradeCmd()) {
+                subBuilder.mergeFrom(getUpgradeCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setUpgradeCmd(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeCommandProto.Type cmdType = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type cmdType_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type.BalancerBandwidthCommand;
+      public boolean hasCmdType() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type getCmdType() {
+        return cmdType_;
+      }
+      public Builder setCmdType(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        cmdType_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearCmdType() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        cmdType_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Type.BalancerBandwidthCommand;
+        onChanged();
+        return this;
+      }
+      
+      // optional .BalancerBandwidthCommandProto balancerCmd = 2;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto balancerCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProtoOrBuilder> balancerCmdBuilder_;
+      public boolean hasBalancerCmd() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto getBalancerCmd() {
+        if (balancerCmdBuilder_ == null) {
+          return balancerCmd_;
+        } else {
+          return balancerCmdBuilder_.getMessage();
+        }
+      }
+      public Builder setBalancerCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto value) {
+        if (balancerCmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          balancerCmd_ = value;
+          onChanged();
+        } else {
+          balancerCmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder setBalancerCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.Builder builderForValue) {
+        if (balancerCmdBuilder_ == null) {
+          balancerCmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          balancerCmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder mergeBalancerCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto value) {
+        if (balancerCmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              balancerCmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.getDefaultInstance()) {
+            balancerCmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.newBuilder(balancerCmd_).mergeFrom(value).buildPartial();
+          } else {
+            balancerCmd_ = value;
+          }
+          onChanged();
+        } else {
+          balancerCmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder clearBalancerCmd() {
+        if (balancerCmdBuilder_ == null) {
+          balancerCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          balancerCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.Builder getBalancerCmdBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getBalancerCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProtoOrBuilder getBalancerCmdOrBuilder() {
+        if (balancerCmdBuilder_ != null) {
+          return balancerCmdBuilder_.getMessageOrBuilder();
+        } else {
+          return balancerCmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProtoOrBuilder> 
+          getBalancerCmdFieldBuilder() {
+        if (balancerCmdBuilder_ == null) {
+          balancerCmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProtoOrBuilder>(
+                  balancerCmd_,
+                  getParentForChildren(),
+                  isClean());
+          balancerCmd_ = null;
+        }
+        return balancerCmdBuilder_;
+      }
+      
+      // optional .BlockCommandProto blkCmd = 3;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto blkCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProtoOrBuilder> blkCmdBuilder_;
+      public boolean hasBlkCmd() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto getBlkCmd() {
+        if (blkCmdBuilder_ == null) {
+          return blkCmd_;
+        } else {
+          return blkCmdBuilder_.getMessage();
+        }
+      }
+      public Builder setBlkCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto value) {
+        if (blkCmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          blkCmd_ = value;
+          onChanged();
+        } else {
+          blkCmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder setBlkCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.Builder builderForValue) {
+        if (blkCmdBuilder_ == null) {
+          blkCmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          blkCmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder mergeBlkCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto value) {
+        if (blkCmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              blkCmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.getDefaultInstance()) {
+            blkCmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.newBuilder(blkCmd_).mergeFrom(value).buildPartial();
+          } else {
+            blkCmd_ = value;
+          }
+          onChanged();
+        } else {
+          blkCmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder clearBlkCmd() {
+        if (blkCmdBuilder_ == null) {
+          blkCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          blkCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.Builder getBlkCmdBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getBlkCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProtoOrBuilder getBlkCmdOrBuilder() {
+        if (blkCmdBuilder_ != null) {
+          return blkCmdBuilder_.getMessageOrBuilder();
+        } else {
+          return blkCmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProtoOrBuilder> 
+          getBlkCmdFieldBuilder() {
+        if (blkCmdBuilder_ == null) {
+          blkCmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProtoOrBuilder>(
+                  blkCmd_,
+                  getParentForChildren(),
+                  isClean());
+          blkCmd_ = null;
+        }
+        return blkCmdBuilder_;
+      }
+      
+      // optional .BlockRecoveryCommndProto recoveryCmd = 4;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto recoveryCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProtoOrBuilder> recoveryCmdBuilder_;
+      public boolean hasRecoveryCmd() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto getRecoveryCmd() {
+        if (recoveryCmdBuilder_ == null) {
+          return recoveryCmd_;
+        } else {
+          return recoveryCmdBuilder_.getMessage();
+        }
+      }
+      public Builder setRecoveryCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto value) {
+        if (recoveryCmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          recoveryCmd_ = value;
+          onChanged();
+        } else {
+          recoveryCmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      public Builder setRecoveryCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.Builder builderForValue) {
+        if (recoveryCmdBuilder_ == null) {
+          recoveryCmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          recoveryCmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      public Builder mergeRecoveryCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto value) {
+        if (recoveryCmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008) &&
+              recoveryCmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.getDefaultInstance()) {
+            recoveryCmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.newBuilder(recoveryCmd_).mergeFrom(value).buildPartial();
+          } else {
+            recoveryCmd_ = value;
+          }
+          onChanged();
+        } else {
+          recoveryCmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      public Builder clearRecoveryCmd() {
+        if (recoveryCmdBuilder_ == null) {
+          recoveryCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.getDefaultInstance();
+          onChanged();
+        } else {
+          recoveryCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.Builder getRecoveryCmdBuilder() {
+        bitField0_ |= 0x00000008;
+        onChanged();
+        return getRecoveryCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProtoOrBuilder getRecoveryCmdOrBuilder() {
+        if (recoveryCmdBuilder_ != null) {
+          return recoveryCmdBuilder_.getMessageOrBuilder();
+        } else {
+          return recoveryCmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProtoOrBuilder> 
+          getRecoveryCmdFieldBuilder() {
+        if (recoveryCmdBuilder_ == null) {
+          recoveryCmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProtoOrBuilder>(
+                  recoveryCmd_,
+                  getParentForChildren(),
+                  isClean());
+          recoveryCmd_ = null;
+        }
+        return recoveryCmdBuilder_;
+      }
+      
+      // optional .FinalizeCommandProto finalizeCmd = 5;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto finalizeCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProtoOrBuilder> finalizeCmdBuilder_;
+      public boolean hasFinalizeCmd() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto getFinalizeCmd() {
+        if (finalizeCmdBuilder_ == null) {
+          return finalizeCmd_;
+        } else {
+          return finalizeCmdBuilder_.getMessage();
+        }
+      }
+      public Builder setFinalizeCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto value) {
+        if (finalizeCmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          finalizeCmd_ = value;
+          onChanged();
+        } else {
+          finalizeCmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000010;
+        return this;
+      }
+      public Builder setFinalizeCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.Builder builderForValue) {
+        if (finalizeCmdBuilder_ == null) {
+          finalizeCmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          finalizeCmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000010;
+        return this;
+      }
+      public Builder mergeFinalizeCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto value) {
+        if (finalizeCmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000010) == 0x00000010) &&
+              finalizeCmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.getDefaultInstance()) {
+            finalizeCmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.newBuilder(finalizeCmd_).mergeFrom(value).buildPartial();
+          } else {
+            finalizeCmd_ = value;
+          }
+          onChanged();
+        } else {
+          finalizeCmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000010;
+        return this;
+      }
+      public Builder clearFinalizeCmd() {
+        if (finalizeCmdBuilder_ == null) {
+          finalizeCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          finalizeCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000010);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.Builder getFinalizeCmdBuilder() {
+        bitField0_ |= 0x00000010;
+        onChanged();
+        return getFinalizeCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProtoOrBuilder getFinalizeCmdOrBuilder() {
+        if (finalizeCmdBuilder_ != null) {
+          return finalizeCmdBuilder_.getMessageOrBuilder();
+        } else {
+          return finalizeCmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProtoOrBuilder> 
+          getFinalizeCmdFieldBuilder() {
+        if (finalizeCmdBuilder_ == null) {
+          finalizeCmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProtoOrBuilder>(
+                  finalizeCmd_,
+                  getParentForChildren(),
+                  isClean());
+          finalizeCmd_ = null;
+        }
+        return finalizeCmdBuilder_;
+      }
+      
+      // optional .KeyUpdateCommandProto keyUpdateCmd = 6;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto keyUpdateCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProtoOrBuilder> keyUpdateCmdBuilder_;
+      public boolean hasKeyUpdateCmd() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto getKeyUpdateCmd() {
+        if (keyUpdateCmdBuilder_ == null) {
+          return keyUpdateCmd_;
+        } else {
+          return keyUpdateCmdBuilder_.getMessage();
+        }
+      }
+      public Builder setKeyUpdateCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto value) {
+        if (keyUpdateCmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          keyUpdateCmd_ = value;
+          onChanged();
+        } else {
+          keyUpdateCmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000020;
+        return this;
+      }
+      public Builder setKeyUpdateCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.Builder builderForValue) {
+        if (keyUpdateCmdBuilder_ == null) {
+          keyUpdateCmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          keyUpdateCmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000020;
+        return this;
+      }
+      public Builder mergeKeyUpdateCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto value) {
+        if (keyUpdateCmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000020) == 0x00000020) &&
+              keyUpdateCmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.getDefaultInstance()) {
+            keyUpdateCmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.newBuilder(keyUpdateCmd_).mergeFrom(value).buildPartial();
+          } else {
+            keyUpdateCmd_ = value;
+          }
+          onChanged();
+        } else {
+          keyUpdateCmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000020;
+        return this;
+      }
+      public Builder clearKeyUpdateCmd() {
+        if (keyUpdateCmdBuilder_ == null) {
+          keyUpdateCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          keyUpdateCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000020);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.Builder getKeyUpdateCmdBuilder() {
+        bitField0_ |= 0x00000020;
+        onChanged();
+        return getKeyUpdateCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProtoOrBuilder getKeyUpdateCmdOrBuilder() {
+        if (keyUpdateCmdBuilder_ != null) {
+          return keyUpdateCmdBuilder_.getMessageOrBuilder();
+        } else {
+          return keyUpdateCmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProtoOrBuilder> 
+          getKeyUpdateCmdFieldBuilder() {
+        if (keyUpdateCmdBuilder_ == null) {
+          keyUpdateCmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProtoOrBuilder>(
+                  keyUpdateCmd_,
+                  getParentForChildren(),
+                  isClean());
+          keyUpdateCmd_ = null;
+        }
+        return keyUpdateCmdBuilder_;
+      }
+      
+      // optional .RegisterCommandProto registerCmd = 7;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto registerCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProtoOrBuilder> registerCmdBuilder_;
+      public boolean hasRegisterCmd() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto getRegisterCmd() {
+        if (registerCmdBuilder_ == null) {
+          return registerCmd_;
+        } else {
+          return registerCmdBuilder_.getMessage();
+        }
+      }
+      public Builder setRegisterCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto value) {
+        if (registerCmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          registerCmd_ = value;
+          onChanged();
+        } else {
+          registerCmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000040;
+        return this;
+      }
+      public Builder setRegisterCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.Builder builderForValue) {
+        if (registerCmdBuilder_ == null) {
+          registerCmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          registerCmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000040;
+        return this;
+      }
+      public Builder mergeRegisterCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto value) {
+        if (registerCmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000040) == 0x00000040) &&
+              registerCmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.getDefaultInstance()) {
+            registerCmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.newBuilder(registerCmd_).mergeFrom(value).buildPartial();
+          } else {
+            registerCmd_ = value;
+          }
+          onChanged();
+        } else {
+          registerCmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000040;
+        return this;
+      }
+      public Builder clearRegisterCmd() {
+        if (registerCmdBuilder_ == null) {
+          registerCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registerCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000040);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.Builder getRegisterCmdBuilder() {
+        bitField0_ |= 0x00000040;
+        onChanged();
+        return getRegisterCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProtoOrBuilder getRegisterCmdOrBuilder() {
+        if (registerCmdBuilder_ != null) {
+          return registerCmdBuilder_.getMessageOrBuilder();
+        } else {
+          return registerCmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProtoOrBuilder> 
+          getRegisterCmdFieldBuilder() {
+        if (registerCmdBuilder_ == null) {
+          registerCmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProtoOrBuilder>(
+                  registerCmd_,
+                  getParentForChildren(),
+                  isClean());
+          registerCmd_ = null;
+        }
+        return registerCmdBuilder_;
+      }
+      
+      // optional .UpgradeCommandProto upgradeCmd = 8;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto upgradeCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder> upgradeCmdBuilder_;
+      public boolean hasUpgradeCmd() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getUpgradeCmd() {
+        if (upgradeCmdBuilder_ == null) {
+          return upgradeCmd_;
+        } else {
+          return upgradeCmdBuilder_.getMessage();
+        }
+      }
+      public Builder setUpgradeCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto value) {
+        if (upgradeCmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          upgradeCmd_ = value;
+          onChanged();
+        } else {
+          upgradeCmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000080;
+        return this;
+      }
+      public Builder setUpgradeCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder builderForValue) {
+        if (upgradeCmdBuilder_ == null) {
+          upgradeCmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          upgradeCmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000080;
+        return this;
+      }
+      public Builder mergeUpgradeCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto value) {
+        if (upgradeCmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000080) == 0x00000080) &&
+              upgradeCmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance()) {
+            upgradeCmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.newBuilder(upgradeCmd_).mergeFrom(value).buildPartial();
+          } else {
+            upgradeCmd_ = value;
+          }
+          onChanged();
+        } else {
+          upgradeCmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000080;
+        return this;
+      }
+      public Builder clearUpgradeCmd() {
+        if (upgradeCmdBuilder_ == null) {
+          upgradeCmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          upgradeCmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000080);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder getUpgradeCmdBuilder() {
+        bitField0_ |= 0x00000080;
+        onChanged();
+        return getUpgradeCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder getUpgradeCmdOrBuilder() {
+        if (upgradeCmdBuilder_ != null) {
+          return upgradeCmdBuilder_.getMessageOrBuilder();
+        } else {
+          return upgradeCmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder> 
+          getUpgradeCmdFieldBuilder() {
+        if (upgradeCmdBuilder_ == null) {
+          upgradeCmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder>(
+                  upgradeCmd_,
+                  getParentForChildren(),
+                  isClean());
+          upgradeCmd_ = null;
+        }
+        return upgradeCmdBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:DatanodeCommandProto)
+    }
+    
+    static {
+      defaultInstance = new DatanodeCommandProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:DatanodeCommandProto)
+  }
+  
+  public interface BalancerBandwidthCommandProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required uint64 bandwidth = 1;
+    boolean hasBandwidth();
+    long getBandwidth();
+  }
+  public static final class BalancerBandwidthCommandProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BalancerBandwidthCommandProtoOrBuilder {
+    // Use BalancerBandwidthCommandProto.newBuilder() to construct.
+    private BalancerBandwidthCommandProto(Builder builder) {
+      super(builder);
+    }
+    private BalancerBandwidthCommandProto(boolean noInit) {}
+    
+    private static final BalancerBandwidthCommandProto defaultInstance;
+    public static BalancerBandwidthCommandProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BalancerBandwidthCommandProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BalancerBandwidthCommandProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BalancerBandwidthCommandProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required uint64 bandwidth = 1;
+    public static final int BANDWIDTH_FIELD_NUMBER = 1;
+    private long bandwidth_;
+    public boolean hasBandwidth() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public long getBandwidth() {
+      return bandwidth_;
+    }
+    
+    private void initFields() {
+      bandwidth_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBandwidth()) {
+        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.writeUInt64(1, bandwidth_);
+      }
+      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
+          .computeUInt64Size(1, bandwidth_);
+      }
+      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.DatanodeProtocolProtos.BalancerBandwidthCommandProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBandwidth() == other.hasBandwidth());
+      if (hasBandwidth()) {
+        result = result && (getBandwidth()
+            == other.getBandwidth());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasBandwidth()) {
+        hash = (37 * hash) + BANDWIDTH_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getBandwidth());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto 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.DatanodeProtocolProtos.BalancerBandwidthCommandProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto 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.DatanodeProtocolProtos.BalancerBandwidthCommandProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto 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.DatanodeProtocolProtos.BalancerBandwidthCommandProto 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.DatanodeProtocolProtos.BalancerBandwidthCommandProto 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.DatanodeProtocolProtos.BalancerBandwidthCommandProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto 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.DatanodeProtocolProtos.BalancerBandwidthCommandProto 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.DatanodeProtocolProtos.BalancerBandwidthCommandProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BalancerBandwidthCommandProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BalancerBandwidthCommandProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.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();
+        bandwidth_ = 0L;
+        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.DatanodeProtocolProtos.BalancerBandwidthCommandProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.bandwidth_ = bandwidth_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.getDefaultInstance()) return this;
+        if (other.hasBandwidth()) {
+          setBandwidth(other.getBandwidth());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBandwidth()) {
+          
+          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;
+              bandwidth_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required uint64 bandwidth = 1;
+      private long bandwidth_ ;
+      public boolean hasBandwidth() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public long getBandwidth() {
+        return bandwidth_;
+      }
+      public Builder setBandwidth(long value) {
+        bitField0_ |= 0x00000001;
+        bandwidth_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearBandwidth() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        bandwidth_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:BalancerBandwidthCommandProto)
+    }
+    
+    static {
+      defaultInstance = new BalancerBandwidthCommandProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BalancerBandwidthCommandProto)
+  }
+  
+  public interface BlockCommandProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required uint32 action = 1;
+    boolean hasAction();
+    int getAction();
+    
+    // required string blockPoolId = 2;
+    boolean hasBlockPoolId();
+    String getBlockPoolId();
+    
+    // repeated .BlockProto blocks = 3;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto> 
+        getBlocksList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto getBlocks(int index);
+    int getBlocksCount();
+    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder> 
+        getBlocksOrBuilderList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder getBlocksOrBuilder(
+        int index);
+    
+    // repeated .DatanodeIDsProto targets = 4;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto> 
+        getTargetsList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto getTargets(int index);
+    int getTargetsCount();
+    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProtoOrBuilder> 
+        getTargetsOrBuilderList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProtoOrBuilder getTargetsOrBuilder(
+        int index);
+  }
+  public static final class BlockCommandProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BlockCommandProtoOrBuilder {
+    // Use BlockCommandProto.newBuilder() to construct.
+    private BlockCommandProto(Builder builder) {
+      super(builder);
+    }
+    private BlockCommandProto(boolean noInit) {}
+    
+    private static final BlockCommandProto defaultInstance;
+    public static BlockCommandProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BlockCommandProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockCommandProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockCommandProto_fieldAccessorTable;
+    }
+    
+    public enum Action
+        implements com.google.protobuf.ProtocolMessageEnum {
+      UNKNOWN(0, 0),
+      TRANSFER(1, 1),
+      INVALIDATE(2, 2),
+      SHUTDOWN(3, 3),
+      ;
+      
+      public static final int UNKNOWN_VALUE = 0;
+      public static final int TRANSFER_VALUE = 1;
+      public static final int INVALIDATE_VALUE = 2;
+      public static final int SHUTDOWN_VALUE = 3;
+      
+      
+      public final int getNumber() { return value; }
+      
+      public static Action valueOf(int value) {
+        switch (value) {
+          case 0: return UNKNOWN;
+          case 1: return TRANSFER;
+          case 2: return INVALIDATE;
+          case 3: return SHUTDOWN;
+          default: return null;
+        }
+      }
+      
+      public static com.google.protobuf.Internal.EnumLiteMap<Action>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<Action>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<Action>() {
+              public Action findValueByNumber(int number) {
+                return Action.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.DatanodeProtocolProtos.BlockCommandProto.getDescriptor().getEnumTypes().get(0);
+      }
+      
+      private static final Action[] VALUES = {
+        UNKNOWN, TRANSFER, INVALIDATE, SHUTDOWN, 
+      };
+      
+      public static Action 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 Action(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+      
+      // @@protoc_insertion_point(enum_scope:BlockCommandProto.Action)
+    }
+    
+    private int bitField0_;
+    // required uint32 action = 1;
+    public static final int ACTION_FIELD_NUMBER = 1;
+    private int action_;
+    public boolean hasAction() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public int getAction() {
+      return action_;
+    }
+    
+    // required string blockPoolId = 2;
+    public static final int BLOCKPOOLID_FIELD_NUMBER = 2;
+    private java.lang.Object blockPoolId_;
+    public boolean hasBlockPoolId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public String getBlockPoolId() {
+      java.lang.Object ref = blockPoolId_;
+      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)) {
+          blockPoolId_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getBlockPoolIdBytes() {
+      java.lang.Object ref = blockPoolId_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        blockPoolId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // repeated .BlockProto blocks = 3;
+    public static final int BLOCKS_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto> blocks_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto> getBlocksList() {
+      return blocks_;
+    }
+    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder> 
+        getBlocksOrBuilderList() {
+      return blocks_;
+    }
+    public int getBlocksCount() {
+      return blocks_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto getBlocks(int index) {
+      return blocks_.get(index);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder getBlocksOrBuilder(
+        int index) {
+      return blocks_.get(index);
+    }
+    
+    // repeated .DatanodeIDsProto targets = 4;
+    public static final int TARGETS_FIELD_NUMBER = 4;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto> targets_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto> getTargetsList() {
+      return targets_;
+    }
+    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProtoOrBuilder> 
+        getTargetsOrBuilderList() {
+      return targets_;
+    }
+    public int getTargetsCount() {
+      return targets_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto getTargets(int index) {
+      return targets_.get(index);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProtoOrBuilder getTargetsOrBuilder(
+        int index) {
+      return targets_.get(index);
+    }
+    
+    private void initFields() {
+      action_ = 0;
+      blockPoolId_ = "";
+      blocks_ = java.util.Collections.emptyList();
+      targets_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasAction()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasBlockPoolId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getBlocksCount(); i++) {
+        if (!getBlocks(i).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.writeUInt32(1, action_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getBlockPoolIdBytes());
+      }
+      for (int i = 0; i < blocks_.size(); i++) {
+        output.writeMessage(3, blocks_.get(i));
+      }
+      for (int i = 0; i < targets_.size(); i++) {
+        output.writeMessage(4, 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
+          .computeUInt32Size(1, action_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getBlockPoolIdBytes());
+      }
+      for (int i = 0; i < blocks_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, blocks_.get(i));
+      }
+      for (int i = 0; i < targets_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, 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.DatanodeProtocolProtos.BlockCommandProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto) obj;
+      
+      boolean result = true;
+      result = result && (hasAction() == other.hasAction());
+      if (hasAction()) {
+        result = result && (getAction()
+            == other.getAction());
+      }
+      result = result && (hasBlockPoolId() == other.hasBlockPoolId());
+      if (hasBlockPoolId()) {
+        result = result && getBlockPoolId()
+            .equals(other.getBlockPoolId());
+      }
+      result = result && getBlocksList()
+          .equals(other.getBlocksList());
+      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 (hasAction()) {
+        hash = (37 * hash) + ACTION_FIELD_NUMBER;
+        hash = (53 * hash) + getAction();
+      }
+      if (hasBlockPoolId()) {
+        hash = (37 * hash) + BLOCKPOOLID_FIELD_NUMBER;
+        hash = (53 * hash) + getBlockPoolId().hashCode();
+      }
+      if (getBlocksCount() > 0) {
+        hash = (37 * hash) + BLOCKS_FIELD_NUMBER;
+        hash = (53 * hash) + getBlocksList().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.DatanodeProtocolProtos.BlockCommandProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto 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.DatanodeProtocolProtos.BlockCommandProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto 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.DatanodeProtocolProtos.BlockCommandProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto 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.DatanodeProtocolProtos.BlockCommandProto 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.DatanodeProtocolProtos.BlockCommandProto 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.DatanodeProtocolProtos.BlockCommandProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto 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.DatanodeProtocolProtos.BlockCommandProto 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.DatanodeProtocolProtos.BlockCommandProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockCommandProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockCommandProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBlocksFieldBuilder();
+          getTargetsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        action_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        blockPoolId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (blocksBuilder_ == null) {
+          blocks_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        } else {
+          blocksBuilder_.clear();
+        }
+        if (targetsBuilder_ == null) {
+          targets_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000008);
+        } 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.DatanodeProtocolProtos.BlockCommandProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.action_ = action_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.blockPoolId_ = blockPoolId_;
+        if (blocksBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            blocks_ = java.util.Collections.unmodifiableList(blocks_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.blocks_ = blocks_;
+        } else {
+          result.blocks_ = blocksBuilder_.build();
+        }
+        if (targetsBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008)) {
+            targets_ = java.util.Collections.unmodifiableList(targets_);
+            bitField0_ = (bitField0_ & ~0x00000008);
+          }
+          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.DatanodeProtocolProtos.BlockCommandProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.getDefaultInstance()) return this;
+        if (other.hasAction()) {
+          setAction(other.getAction());
+        }
+        if (other.hasBlockPoolId()) {
+          setBlockPoolId(other.getBlockPoolId());
+        }
+        if (blocksBuilder_ == null) {
+          if (!other.blocks_.isEmpty()) {
+            if (blocks_.isEmpty()) {
+              blocks_ = other.blocks_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+            } else {
+              ensureBlocksIsMutable();
+              blocks_.addAll(other.blocks_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.blocks_.isEmpty()) {
+            if (blocksBuilder_.isEmpty()) {
+              blocksBuilder_.dispose();
+              blocksBuilder_ = null;
+              blocks_ = other.blocks_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+              blocksBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getBlocksFieldBuilder() : null;
+            } else {
+              blocksBuilder_.addAllMessages(other.blocks_);
+            }
+          }
+        }
+        if (targetsBuilder_ == null) {
+          if (!other.targets_.isEmpty()) {
+            if (targets_.isEmpty()) {
+              targets_ = other.targets_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+            } else {
+              ensureTargetsIsMutable();
+              targets_.addAll(other.targets_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.targets_.isEmpty()) {
+            if (targetsBuilder_.isEmpty()) {
+              targetsBuilder_.dispose();
+              targetsBuilder_ = null;
+              targets_ = other.targets_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+              targetsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTargetsFieldBuilder() : null;
+            } else {
+              targetsBuilder_.addAllMessages(other.targets_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasAction()) {
+          
+          return false;
+        }
+        if (!hasBlockPoolId()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getBlocksCount(); i++) {
+          if (!getBlocks(i).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 8: {
+              bitField0_ |= 0x00000001;
+              action_ = input.readUInt32();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              blockPoolId_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.newBuilder();
+              input.readMessage(subBuilder, extensionRegistry);
+              addBlocks(subBuilder.buildPartial());
+              break;
+            }
+            case 34: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.newBuilder();
+              input.readMessage(subBuilder, extensionRegistry);
+              addTargets(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required uint32 action = 1;
+      private int action_ ;
+      public boolean hasAction() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public int getAction() {
+        return action_;
+      }
+      public Builder setAction(int value) {
+        bitField0_ |= 0x00000001;
+        action_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearAction() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        action_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // required string blockPoolId = 2;
+      private java.lang.Object blockPoolId_ = "";
+      public boolean hasBlockPoolId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public String getBlockPoolId() {
+        java.lang.Object ref = blockPoolId_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          blockPoolId_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setBlockPoolId(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        blockPoolId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearBlockPoolId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        blockPoolId_ = getDefaultInstance().getBlockPoolId();
+        onChanged();
+        return this;
+      }
+      void setBlockPoolId(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000002;
+        blockPoolId_ = value;
+        onChanged();
+      }
+      
+      // repeated .BlockProto blocks = 3;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto> blocks_ =
+        java.util.Collections.emptyList();
+      private void ensureBlocksIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          blocks_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto>(blocks_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+      
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder> blocksBuilder_;
+      
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto> getBlocksList() {
+        if (blocksBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(blocks_);
+        } else {
+          return blocksBuilder_.getMessageList();
+        }
+      }
+      public int getBlocksCount() {
+        if (blocksBuilder_ == null) {
+          return blocks_.size();
+        } else {
+          return blocksBuilder_.getCount();
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto getBlocks(int index) {
+        if (blocksBuilder_ == null) {
+          return blocks_.get(index);
+        } else {
+          return blocksBuilder_.getMessage(index);
+        }
+      }
+      public Builder setBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.set(index, value);
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      public Builder setBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addBlocks(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.add(value);
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.add(index, value);
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.add(builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addAllBlocks(
+          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto> values) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          super.addAll(values, blocks_);
+          onChanged();
+        } else {
+          blocksBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      public Builder clearBlocks() {
+        if (blocksBuilder_ == null) {
+          blocks_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+          onChanged();
+        } else {
+          blocksBuilder_.clear();
+        }
+        return this;
+      }
+      public Builder removeBlocks(int index) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.remove(index);
+          onChanged();
+        } else {
+          blocksBuilder_.remove(index);
+        }
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder getBlocksBuilder(
+          int index) {
+        return getBlocksFieldBuilder().getBuilder(index);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder getBlocksOrBuilder(
+          int index) {
+        if (blocksBuilder_ == null) {
+          return blocks_.get(index);  } else {
+          return blocksBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder> 
+           getBlocksOrBuilderList() {
+        if (blocksBuilder_ != null) {
+          return blocksBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(blocks_);
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder addBlocksBuilder() {
+        return getBlocksFieldBuilder().addBuilder(
+            org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance());
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder addBlocksBuilder(
+          int index) {
+        return getBlocksFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance());
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder> 
+           getBlocksBuilderList() {
+        return getBlocksFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder> 
+          getBlocksFieldBuilder() {
+        if (blocksBuilder_ == null) {
+          blocksBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder>(
+                  blocks_,
+                  ((bitField0_ & 0x00000004) == 0x00000004),
+                  getParentForChildren(),
+                  isClean());
+          blocks_ = null;
+        }
+        return blocksBuilder_;
+      }
+      
+      // repeated .DatanodeIDsProto targets = 4;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto> targets_ =
+        java.util.Collections.emptyList();
+      private void ensureTargetsIsMutable() {
+        if (!((bitField0_ & 0x00000008) == 0x00000008)) {
+          targets_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto>(targets_);
+          bitField0_ |= 0x00000008;
+         }
+      }
+      
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProtoOrBuilder> targetsBuilder_;
+      
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto> 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.DatanodeIDsProto 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.DatanodeIDsProto 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.DatanodeIDsProto.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.DatanodeIDsProto 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.DatanodeIDsProto 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.DatanodeIDsProto.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.DatanodeIDsProto.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.DatanodeIDsProto> 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_ & ~0x00000008);
+          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.DatanodeIDsProto.Builder getTargetsBuilder(
+          int index) {
+        return getTargetsFieldBuilder().getBuilder(index);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProtoOrBuilder 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.DatanodeIDsProtoOrBuilder> 
+           getTargetsOrBuilderList() {
+        if (targetsBuilder_ != null) {
+          return targetsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(targets_);
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.Builder addTargetsBuilder() {
+        return getTargetsFieldBuilder().addBuilder(
+            org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.getDefaultInstance());
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.Builder addTargetsBuilder(
+          int index) {
+        return getTargetsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.getDefaultInstance());
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.Builder> 
+           getTargetsBuilderList() {
+        return getTargetsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProtoOrBuilder> 
+          getTargetsFieldBuilder() {
+        if (targetsBuilder_ == null) {
+          targetsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDsProtoOrBuilder>(
+                  targets_,
+                  ((bitField0_ & 0x00000008) == 0x00000008),
+                  getParentForChildren(),
+                  isClean());
+          targets_ = null;
+        }
+        return targetsBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:BlockCommandProto)
+    }
+    
+    static {
+      defaultInstance = new BlockCommandProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BlockCommandProto)
+  }
+  
+  public interface BlockRecoveryCommndProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // repeated .RecoveringBlockProto blocks = 1;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto> 
+        getBlocksList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto getBlocks(int index);
+    int getBlocksCount();
+    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder> 
+        getBlocksOrBuilderList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder getBlocksOrBuilder(
+        int index);
+  }
+  public static final class BlockRecoveryCommndProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BlockRecoveryCommndProtoOrBuilder {
+    // Use BlockRecoveryCommndProto.newBuilder() to construct.
+    private BlockRecoveryCommndProto(Builder builder) {
+      super(builder);
+    }
+    private BlockRecoveryCommndProto(boolean noInit) {}
+    
+    private static final BlockRecoveryCommndProto defaultInstance;
+    public static BlockRecoveryCommndProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BlockRecoveryCommndProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockRecoveryCommndProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockRecoveryCommndProto_fieldAccessorTable;
+    }
+    
+    // repeated .RecoveringBlockProto blocks = 1;
+    public static final int BLOCKS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto> blocks_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto> getBlocksList() {
+      return blocks_;
+    }
+    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder> 
+        getBlocksOrBuilderList() {
+      return blocks_;
+    }
+    public int getBlocksCount() {
+      return blocks_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto getBlocks(int index) {
+      return blocks_.get(index);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder getBlocksOrBuilder(
+        int index) {
+      return blocks_.get(index);
+    }
+    
+    private void initFields() {
+      blocks_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      for (int i = 0; i < getBlocksCount(); i++) {
+        if (!getBlocks(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < blocks_.size(); i++) {
+        output.writeMessage(1, blocks_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      for (int i = 0; i < blocks_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, blocks_.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.DatanodeProtocolProtos.BlockRecoveryCommndProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto) obj;
+      
+      boolean result = true;
+      result = result && getBlocksList()
+          .equals(other.getBlocksList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (getBlocksCount() > 0) {
+        hash = (37 * hash) + BLOCKS_FIELD_NUMBER;
+        hash = (53 * hash) + getBlocksList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto 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.DatanodeProtocolProtos.BlockRecoveryCommndProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto 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.DatanodeProtocolProtos.BlockRecoveryCommndProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto 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.DatanodeProtocolProtos.BlockRecoveryCommndProto 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.DatanodeProtocolProtos.BlockRecoveryCommndProto 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.DatanodeProtocolProtos.BlockRecoveryCommndProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto 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.DatanodeProtocolProtos.BlockRecoveryCommndProto 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.DatanodeProtocolProtos.BlockRecoveryCommndProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockRecoveryCommndProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockRecoveryCommndProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBlocksFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (blocksBuilder_ == null) {
+          blocks_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          blocksBuilder_.clear();
+        }
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto(this);
+        int from_bitField0_ = bitField0_;
+        if (blocksBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            blocks_ = java.util.Collections.unmodifiableList(blocks_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.blocks_ = blocks_;
+        } else {
+          result.blocks_ = blocksBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.getDefaultInstance()) return this;
+        if (blocksBuilder_ == null) {
+          if (!other.blocks_.isEmpty()) {
+            if (blocks_.isEmpty()) {
+              blocks_ = other.blocks_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureBlocksIsMutable();
+              blocks_.addAll(other.blocks_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.blocks_.isEmpty()) {
+            if (blocksBuilder_.isEmpty()) {
+              blocksBuilder_.dispose();
+              blocksBuilder_ = null;
+              blocks_ = other.blocks_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              blocksBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getBlocksFieldBuilder() : null;
+            } else {
+              blocksBuilder_.addAllMessages(other.blocks_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        for (int i = 0; i < getBlocksCount(); i++) {
+          if (!getBlocks(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.HdfsProtos.RecoveringBlockProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.newBuilder();
+              input.readMessage(subBuilder, extensionRegistry);
+              addBlocks(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // repeated .RecoveringBlockProto blocks = 1;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto> blocks_ =
+        java.util.Collections.emptyList();
+      private void ensureBlocksIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          blocks_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto>(blocks_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+      
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder> blocksBuilder_;
+      
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto> getBlocksList() {
+        if (blocksBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(blocks_);
+        } else {
+          return blocksBuilder_.getMessageList();
+        }
+      }
+      public int getBlocksCount() {
+        if (blocksBuilder_ == null) {
+          return blocks_.size();
+        } else {
+          return blocksBuilder_.getCount();
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto getBlocks(int index) {
+        if (blocksBuilder_ == null) {
+          return blocks_.get(index);
+        } else {
+          return blocksBuilder_.getMessage(index);
+        }
+      }
+      public Builder setBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.set(index, value);
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      public Builder setBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addBlocks(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.add(value);
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.add(index, value);
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.add(builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addAllBlocks(
+          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto> values) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          super.addAll(values, blocks_);
+          onChanged();
+        } else {
+          blocksBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      public Builder clearBlocks() {
+        if (blocksBuilder_ == null) {
+          blocks_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          blocksBuilder_.clear();
+        }
+        return this;
+      }
+      public Builder removeBlocks(int index) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.remove(index);
+          onChanged();
+        } else {
+          blocksBuilder_.remove(index);
+        }
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder getBlocksBuilder(
+          int index) {
+        return getBlocksFieldBuilder().getBuilder(index);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder getBlocksOrBuilder(
+          int index) {
+        if (blocksBuilder_ == null) {
+          return blocks_.get(index);  } else {
+          return blocksBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder> 
+           getBlocksOrBuilderList() {
+        if (blocksBuilder_ != null) {
+          return blocksBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(blocks_);
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder addBlocksBuilder() {
+        return getBlocksFieldBuilder().addBuilder(
+            org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.getDefaultInstance());
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder addBlocksBuilder(
+          int index) {
+        return getBlocksFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.getDefaultInstance());
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder> 
+           getBlocksBuilderList() {
+        return getBlocksFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder> 
+          getBlocksFieldBuilder() {
+        if (blocksBuilder_ == null) {
+          blocksBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder>(
+                  blocks_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          blocks_ = null;
+        }
+        return blocksBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:BlockRecoveryCommndProto)
+    }
+    
+    static {
+      defaultInstance = new BlockRecoveryCommndProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BlockRecoveryCommndProto)
+  }
+  
+  public interface FinalizeCommandProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required string blockPoolId = 1;
+    boolean hasBlockPoolId();
+    String getBlockPoolId();
+  }
+  public static final class FinalizeCommandProto extends
+      com.google.protobuf.GeneratedMessage
+      implements FinalizeCommandProtoOrBuilder {
+    // Use FinalizeCommandProto.newBuilder() to construct.
+    private FinalizeCommandProto(Builder builder) {
+      super(builder);
+    }
+    private FinalizeCommandProto(boolean noInit) {}
+    
+    private static final FinalizeCommandProto defaultInstance;
+    public static FinalizeCommandProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public FinalizeCommandProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_FinalizeCommandProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_FinalizeCommandProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required string blockPoolId = 1;
+    public static final int BLOCKPOOLID_FIELD_NUMBER = 1;
+    private java.lang.Object blockPoolId_;
+    public boolean hasBlockPoolId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public String getBlockPoolId() {
+      java.lang.Object ref = blockPoolId_;
+      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)) {
+          blockPoolId_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getBlockPoolIdBytes() {
+      java.lang.Object ref = blockPoolId_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        blockPoolId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    private void initFields() {
+      blockPoolId_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlockPoolId()) {
+        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, getBlockPoolIdBytes());
+      }
+      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, getBlockPoolIdBytes());
+      }
+      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.DatanodeProtocolProtos.FinalizeCommandProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlockPoolId() == other.hasBlockPoolId());
+      if (hasBlockPoolId()) {
+        result = result && getBlockPoolId()
+            .equals(other.getBlockPoolId());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasBlockPoolId()) {
+        hash = (37 * hash) + BLOCKPOOLID_FIELD_NUMBER;
+        hash = (53 * hash) + getBlockPoolId().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto 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.DatanodeProtocolProtos.FinalizeCommandProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto 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.DatanodeProtocolProtos.FinalizeCommandProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto 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.DatanodeProtocolProtos.FinalizeCommandProto 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.DatanodeProtocolProtos.FinalizeCommandProto 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.DatanodeProtocolProtos.FinalizeCommandProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto 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.DatanodeProtocolProtos.FinalizeCommandProto 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.DatanodeProtocolProtos.FinalizeCommandProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_FinalizeCommandProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_FinalizeCommandProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.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();
+        blockPoolId_ = "";
+        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.DatanodeProtocolProtos.FinalizeCommandProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.blockPoolId_ = blockPoolId_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.getDefaultInstance()) return this;
+        if (other.hasBlockPoolId()) {
+          setBlockPoolId(other.getBlockPoolId());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBlockPoolId()) {
+          
+          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;
+              blockPoolId_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required string blockPoolId = 1;
+      private java.lang.Object blockPoolId_ = "";
+      public boolean hasBlockPoolId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public String getBlockPoolId() {
+        java.lang.Object ref = blockPoolId_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          blockPoolId_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setBlockPoolId(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        blockPoolId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearBlockPoolId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        blockPoolId_ = getDefaultInstance().getBlockPoolId();
+        onChanged();
+        return this;
+      }
+      void setBlockPoolId(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000001;
+        blockPoolId_ = value;
+        onChanged();
+      }
+      
+      // @@protoc_insertion_point(builder_scope:FinalizeCommandProto)
+    }
+    
+    static {
+      defaultInstance = new FinalizeCommandProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:FinalizeCommandProto)
+  }
+  
+  public interface KeyUpdateCommandProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .ExportedBlockKeysProto keys = 1;
+    boolean hasKeys();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto getKeys();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder getKeysOrBuilder();
+  }
+  public static final class KeyUpdateCommandProto extends
+      com.google.protobuf.GeneratedMessage
+      implements KeyUpdateCommandProtoOrBuilder {
+    // Use KeyUpdateCommandProto.newBuilder() to construct.
+    private KeyUpdateCommandProto(Builder builder) {
+      super(builder);
+    }
+    private KeyUpdateCommandProto(boolean noInit) {}
+    
+    private static final KeyUpdateCommandProto defaultInstance;
+    public static KeyUpdateCommandProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public KeyUpdateCommandProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_KeyUpdateCommandProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_KeyUpdateCommandProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .ExportedBlockKeysProto keys = 1;
+    public static final int KEYS_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto keys_;
+    public boolean hasKeys() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto getKeys() {
+      return keys_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder getKeysOrBuilder() {
+      return keys_;
+    }
+    
+    private void initFields() {
+      keys_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasKeys()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getKeys().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, keys_);
+      }
+      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, keys_);
+      }
+      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.DatanodeProtocolProtos.KeyUpdateCommandProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto) obj;
+      
+      boolean result = true;
+      result = result && (hasKeys() == other.hasKeys());
+      if (hasKeys()) {
+        result = result && getKeys()
+            .equals(other.getKeys());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasKeys()) {
+        hash = (37 * hash) + KEYS_FIELD_NUMBER;
+        hash = (53 * hash) + getKeys().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto 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.DatanodeProtocolProtos.KeyUpdateCommandProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto 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.DatanodeProtocolProtos.KeyUpdateCommandProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto 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.DatanodeProtocolProtos.KeyUpdateCommandProto 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.DatanodeProtocolProtos.KeyUpdateCommandProto 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.DatanodeProtocolProtos.KeyUpdateCommandProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto 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.DatanodeProtocolProtos.KeyUpdateCommandProto 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.DatanodeProtocolProtos.KeyUpdateCommandProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_KeyUpdateCommandProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_KeyUpdateCommandProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getKeysFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (keysBuilder_ == null) {
+          keys_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance();
+        } else {
+          keysBuilder_.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.DatanodeProtocolProtos.KeyUpdateCommandProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (keysBuilder_ == null) {
+          result.keys_ = keys_;
+        } else {
+          result.keys_ = keysBuilder_.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.DatanodeProtocolProtos.KeyUpdateCommandProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.getDefaultInstance()) return this;
+        if (other.hasKeys()) {
+          mergeKeys(other.getKeys());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasKeys()) {
+          
+          return false;
+        }
+        if (!getKeys().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.ExportedBlockKeysProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.newBuilder();
+              if (hasKeys()) {
+                subBuilder.mergeFrom(getKeys());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setKeys(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .ExportedBlockKeysProto keys = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto keys_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder> keysBuilder_;
+      public boolean hasKeys() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto getKeys() {
+        if (keysBuilder_ == null) {
+          return keys_;
+        } else {
+          return keysBuilder_.getMessage();
+        }
+      }
+      public Builder setKeys(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto value) {
+        if (keysBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          keys_ = value;
+          onChanged();
+        } else {
+          keysBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setKeys(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder builderForValue) {
+        if (keysBuilder_ == null) {
+          keys_ = builderForValue.build();
+          onChanged();
+        } else {
+          keysBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeKeys(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto value) {
+        if (keysBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              keys_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance()) {
+            keys_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.newBuilder(keys_).mergeFrom(value).buildPartial();
+          } else {
+            keys_ = value;
+          }
+          onChanged();
+        } else {
+          keysBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearKeys() {
+        if (keysBuilder_ == null) {
+          keys_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.getDefaultInstance();
+          onChanged();
+        } else {
+          keysBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder getKeysBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getKeysFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder getKeysOrBuilder() {
+        if (keysBuilder_ != null) {
+          return keysBuilder_.getMessageOrBuilder();
+        } else {
+          return keys_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder> 
+          getKeysFieldBuilder() {
+        if (keysBuilder_ == null) {
+          keysBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProtoOrBuilder>(
+                  keys_,
+                  getParentForChildren(),
+                  isClean());
+          keys_ = null;
+        }
+        return keysBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:KeyUpdateCommandProto)
+    }
+    
+    static {
+      defaultInstance = new KeyUpdateCommandProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:KeyUpdateCommandProto)
+  }
+  
+  public interface RegisterCommandProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class RegisterCommandProto extends
+      com.google.protobuf.GeneratedMessage
+      implements RegisterCommandProtoOrBuilder {
+    // Use RegisterCommandProto.newBuilder() to construct.
+    private RegisterCommandProto(Builder builder) {
+      super(builder);
+    }
+    private RegisterCommandProto(boolean noInit) {}
+    
+    private static final RegisterCommandProto defaultInstance;
+    public static RegisterCommandProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public RegisterCommandProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterCommandProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterCommandProto_fieldAccessorTable;
+    }
+    
+    private void initFields() {
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      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.DatanodeProtocolProtos.RegisterCommandProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto) obj;
+      
+      boolean result = true;
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto 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.DatanodeProtocolProtos.RegisterCommandProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto 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.DatanodeProtocolProtos.RegisterCommandProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto 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.DatanodeProtocolProtos.RegisterCommandProto 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.DatanodeProtocolProtos.RegisterCommandProto 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.DatanodeProtocolProtos.RegisterCommandProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto 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.DatanodeProtocolProtos.RegisterCommandProto 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.DatanodeProtocolProtos.RegisterCommandProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterCommandProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterCommandProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.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();
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        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;
+            }
+          }
+        }
+      }
+      
+      
+      // @@protoc_insertion_point(builder_scope:RegisterCommandProto)
+    }
+    
+    static {
+      defaultInstance = new RegisterCommandProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:RegisterCommandProto)
+  }
+  
+  public interface UpgradeCommandProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required uint32 action = 1;
+    boolean hasAction();
+    int getAction();
+    
+    // required uint32 version = 2;
+    boolean hasVersion();
+    int getVersion();
+    
+    // required uint32 upgradeStatus = 3;
+    boolean hasUpgradeStatus();
+    int getUpgradeStatus();
+  }
+  public static final class UpgradeCommandProto extends
+      com.google.protobuf.GeneratedMessage
+      implements UpgradeCommandProtoOrBuilder {
+    // Use UpgradeCommandProto.newBuilder() to construct.
+    private UpgradeCommandProto(Builder builder) {
+      super(builder);
+    }
+    private UpgradeCommandProto(boolean noInit) {}
+    
+    private static final UpgradeCommandProto defaultInstance;
+    public static UpgradeCommandProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public UpgradeCommandProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_UpgradeCommandProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_UpgradeCommandProto_fieldAccessorTable;
+    }
+    
+    public enum Action
+        implements com.google.protobuf.ProtocolMessageEnum {
+      UNKNOWN(0, 0),
+      REPORT_STATUS(1, 100),
+      START_UPGRADE(2, 101),
+      ;
+      
+      public static final int UNKNOWN_VALUE = 0;
+      public static final int REPORT_STATUS_VALUE = 100;
+      public static final int START_UPGRADE_VALUE = 101;
+      
+      
+      public final int getNumber() { return value; }
+      
+      public static Action valueOf(int value) {
+        switch (value) {
+          case 0: return UNKNOWN;
+          case 100: return REPORT_STATUS;
+          case 101: return START_UPGRADE;
+          default: return null;
+        }
+      }
+      
+      public static com.google.protobuf.Internal.EnumLiteMap<Action>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<Action>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<Action>() {
+              public Action findValueByNumber(int number) {
+                return Action.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.DatanodeProtocolProtos.UpgradeCommandProto.getDescriptor().getEnumTypes().get(0);
+      }
+      
+      private static final Action[] VALUES = {
+        UNKNOWN, REPORT_STATUS, START_UPGRADE, 
+      };
+      
+      public static Action 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 Action(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+      
+      // @@protoc_insertion_point(enum_scope:UpgradeCommandProto.Action)
+    }
+    
+    private int bitField0_;
+    // required uint32 action = 1;
+    public static final int ACTION_FIELD_NUMBER = 1;
+    private int action_;
+    public boolean hasAction() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public int getAction() {
+      return action_;
+    }
+    
+    // required uint32 version = 2;
+    public static final int VERSION_FIELD_NUMBER = 2;
+    private int version_;
+    public boolean hasVersion() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public int getVersion() {
+      return version_;
+    }
+    
+    // required uint32 upgradeStatus = 3;
+    public static final int UPGRADESTATUS_FIELD_NUMBER = 3;
+    private int upgradeStatus_;
+    public boolean hasUpgradeStatus() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public int getUpgradeStatus() {
+      return upgradeStatus_;
+    }
+    
+    private void initFields() {
+      action_ = 0;
+      version_ = 0;
+      upgradeStatus_ = 0;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasAction()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasVersion()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasUpgradeStatus()) {
+        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, action_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt32(2, version_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt32(3, upgradeStatus_);
+      }
+      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, action_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(2, version_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(3, upgradeStatus_);
+      }
+      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.DatanodeProtocolProtos.UpgradeCommandProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto) obj;
+      
+      boolean result = true;
+      result = result && (hasAction() == other.hasAction());
+      if (hasAction()) {
+        result = result && (getAction()
+            == other.getAction());
+      }
+      result = result && (hasVersion() == other.hasVersion());
+      if (hasVersion()) {
+        result = result && (getVersion()
+            == other.getVersion());
+      }
+      result = result && (hasUpgradeStatus() == other.hasUpgradeStatus());
+      if (hasUpgradeStatus()) {
+        result = result && (getUpgradeStatus()
+            == other.getUpgradeStatus());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasAction()) {
+        hash = (37 * hash) + ACTION_FIELD_NUMBER;
+        hash = (53 * hash) + getAction();
+      }
+      if (hasVersion()) {
+        hash = (37 * hash) + VERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getVersion();
+      }
+      if (hasUpgradeStatus()) {
+        hash = (37 * hash) + UPGRADESTATUS_FIELD_NUMBER;
+        hash = (53 * hash) + getUpgradeStatus();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto 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.DatanodeProtocolProtos.UpgradeCommandProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto 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.DatanodeProtocolProtos.UpgradeCommandProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto 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.DatanodeProtocolProtos.UpgradeCommandProto 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.DatanodeProtocolProtos.UpgradeCommandProto 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.DatanodeProtocolProtos.UpgradeCommandProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto 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.DatanodeProtocolProtos.UpgradeCommandProto 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.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_UpgradeCommandProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_UpgradeCommandProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.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();
+        action_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        version_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        upgradeStatus_ = 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.DatanodeProtocolProtos.UpgradeCommandProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.action_ = action_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.version_ = version_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.upgradeStatus_ = upgradeStatus_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance()) return this;
+        if (other.hasAction()) {
+          setAction(other.getAction());
+        }
+        if (other.hasVersion()) {
+          setVersion(other.getVersion());
+        }
+        if (other.hasUpgradeStatus()) {
+          setUpgradeStatus(other.getUpgradeStatus());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasAction()) {
+          
+          return false;
+        }
+        if (!hasVersion()) {
+          
+          return false;
+        }
+        if (!hasUpgradeStatus()) {
+          
+          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;
+              action_ = input.readUInt32();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              version_ = input.readUInt32();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              upgradeStatus_ = input.readUInt32();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required uint32 action = 1;
+      private int action_ ;
+      public boolean hasAction() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public int getAction() {
+        return action_;
+      }
+      public Builder setAction(int value) {
+        bitField0_ |= 0x00000001;
+        action_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearAction() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        action_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // required uint32 version = 2;
+      private int version_ ;
+      public boolean hasVersion() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public int getVersion() {
+        return version_;
+      }
+      public Builder setVersion(int value) {
+        bitField0_ |= 0x00000002;
+        version_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearVersion() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        version_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // required uint32 upgradeStatus = 3;
+      private int upgradeStatus_ ;
+      public boolean hasUpgradeStatus() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public int getUpgradeStatus() {
+        return upgradeStatus_;
+      }
+      public Builder setUpgradeStatus(int value) {
+        bitField0_ |= 0x00000004;
+        upgradeStatus_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearUpgradeStatus() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        upgradeStatus_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:UpgradeCommandProto)
+    }
+    
+    static {
+      defaultInstance = new UpgradeCommandProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:UpgradeCommandProto)
+  }
+  
+  public interface RegisterDatanodeRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+  }
+  public static final class RegisterDatanodeRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements RegisterDatanodeRequestProtoOrBuilder {
+    // Use RegisterDatanodeRequestProto.newBuilder() to construct.
+    private RegisterDatanodeRequestProto(Builder builder) {
+      super(builder);
+    }
+    private RegisterDatanodeRequestProto(boolean noInit) {}
+    
+    private static final RegisterDatanodeRequestProto defaultInstance;
+    public static RegisterDatanodeRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public RegisterDatanodeRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterDatanodeRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterDatanodeRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .DatanodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasRegistration()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegistration().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, registration_);
+      }
+      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, registration_);
+      }
+      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.DatanodeProtocolProtos.RegisterDatanodeRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasRegistration() == other.hasRegistration());
+      if (hasRegistration()) {
+        result = result && getRegistration()
+            .equals(other.getRegistration());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasRegistration()) {
+        hash = (37 * hash) + REGISTRATION_FIELD_NUMBER;
+        hash = (53 * hash) + getRegistration().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto 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.DatanodeProtocolProtos.RegisterDatanodeRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto 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.DatanodeProtocolProtos.RegisterDatanodeRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto 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.DatanodeProtocolProtos.RegisterDatanodeRequestProto 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.DatanodeProtocolProtos.RegisterDatanodeRequestProto 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.DatanodeProtocolProtos.RegisterDatanodeRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto 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.DatanodeProtocolProtos.RegisterDatanodeRequestProto 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.DatanodeProtocolProtos.RegisterDatanodeRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterDatanodeRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterDatanodeRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getRegistrationFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+        } else {
+          registrationBuilder_.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.DatanodeProtocolProtos.RegisterDatanodeRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (registrationBuilder_ == null) {
+          result.registration_ = registration_;
+        } else {
+          result.registration_ = registrationBuilder_.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.DatanodeProtocolProtos.RegisterDatanodeRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto.getDefaultInstance()) return this;
+        if (other.hasRegistration()) {
+          mergeRegistration(other.getRegistration());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasRegistration()) {
+          
+          return false;
+        }
+        if (!getRegistration().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.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          registration_ = value;
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setRegistration(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder builderForValue) {
+        if (registrationBuilder_ == null) {
+          registration_ = builderForValue.build();
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder(registration_).mergeFrom(value).buildPartial();
+          } else {
+            registration_ = value;
+          }
+          onChanged();
+        } else {
+          registrationBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearRegistration() {
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:RegisterDatanodeRequestProto)
+    }
+    
+    static {
+      defaultInstance = new RegisterDatanodeRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:RegisterDatanodeRequestProto)
+  }
+  
+  public interface RegisterDatanodeResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+  }
+  public static final class RegisterDatanodeResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements RegisterDatanodeResponseProtoOrBuilder {
+    // Use RegisterDatanodeResponseProto.newBuilder() to construct.
+    private RegisterDatanodeResponseProto(Builder builder) {
+      super(builder);
+    }
+    private RegisterDatanodeResponseProto(boolean noInit) {}
+    
+    private static final RegisterDatanodeResponseProto defaultInstance;
+    public static RegisterDatanodeResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public RegisterDatanodeResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterDatanodeResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterDatanodeResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .DatanodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasRegistration()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegistration().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, registration_);
+      }
+      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, registration_);
+      }
+      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.DatanodeProtocolProtos.RegisterDatanodeResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasRegistration() == other.hasRegistration());
+      if (hasRegistration()) {
+        result = result && getRegistration()
+            .equals(other.getRegistration());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasRegistration()) {
+        hash = (37 * hash) + REGISTRATION_FIELD_NUMBER;
+        hash = (53 * hash) + getRegistration().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto 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.DatanodeProtocolProtos.RegisterDatanodeResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto 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.DatanodeProtocolProtos.RegisterDatanodeResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto 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.DatanodeProtocolProtos.RegisterDatanodeResponseProto 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.DatanodeProtocolProtos.RegisterDatanodeResponseProto 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.DatanodeProtocolProtos.RegisterDatanodeResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto 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.DatanodeProtocolProtos.RegisterDatanodeResponseProto 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.DatanodeProtocolProtos.RegisterDatanodeResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterDatanodeResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_RegisterDatanodeResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getRegistrationFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+        } else {
+          registrationBuilder_.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.DatanodeProtocolProtos.RegisterDatanodeResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (registrationBuilder_ == null) {
+          result.registration_ = registration_;
+        } else {
+          result.registration_ = registrationBuilder_.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.DatanodeProtocolProtos.RegisterDatanodeResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.getDefaultInstance()) return this;
+        if (other.hasRegistration()) {
+          mergeRegistration(other.getRegistration());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasRegistration()) {
+          
+          return false;
+        }
+        if (!getRegistration().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.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          registration_ = value;
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setRegistration(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder builderForValue) {
+        if (registrationBuilder_ == null) {
+          registration_ = builderForValue.build();
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder(registration_).mergeFrom(value).buildPartial();
+          } else {
+            registration_ = value;
+          }
+          onChanged();
+        } else {
+          registrationBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearRegistration() {
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:RegisterDatanodeResponseProto)
+    }
+    
+    static {
+      defaultInstance = new RegisterDatanodeResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:RegisterDatanodeResponseProto)
+  }
+  
+  public interface HeartbeatRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+    
+    // required uint64 capacity = 2;
+    boolean hasCapacity();
+    long getCapacity();
+    
+    // required uint64 dfsUsed = 3;
+    boolean hasDfsUsed();
+    long getDfsUsed();
+    
+    // required uint64 remaining = 4;
+    boolean hasRemaining();
+    long getRemaining();
+    
+    // required uint64 blockPoolUsed = 5;
+    boolean hasBlockPoolUsed();
+    long getBlockPoolUsed();
+    
+    // required uint32 xmitsInProgress = 6;
+    boolean hasXmitsInProgress();
+    int getXmitsInProgress();
+    
+    // required uint32 xceiverCount = 7;
+    boolean hasXceiverCount();
+    int getXceiverCount();
+    
+    // required uint32 failedVolumes = 8;
+    boolean hasFailedVolumes();
+    int getFailedVolumes();
+  }
+  public static final class HeartbeatRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements HeartbeatRequestProtoOrBuilder {
+    // Use HeartbeatRequestProto.newBuilder() to construct.
+    private HeartbeatRequestProto(Builder builder) {
+      super(builder);
+    }
+    private HeartbeatRequestProto(boolean noInit) {}
+    
+    private static final HeartbeatRequestProto defaultInstance;
+    public static HeartbeatRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public HeartbeatRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_HeartbeatRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_HeartbeatRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .DatanodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    // required 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_;
+    }
+    
+    // required 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_;
+    }
+    
+    // required 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_;
+    }
+    
+    // required 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_;
+    }
+    
+    // required uint32 xmitsInProgress = 6;
+    public static final int XMITSINPROGRESS_FIELD_NUMBER = 6;
+    private int xmitsInProgress_;
+    public boolean hasXmitsInProgress() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    public int getXmitsInProgress() {
+      return xmitsInProgress_;
+    }
+    
+    // required 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_;
+    }
+    
+    // required uint32 failedVolumes = 8;
+    public static final int FAILEDVOLUMES_FIELD_NUMBER = 8;
+    private int failedVolumes_;
+    public boolean hasFailedVolumes() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    public int getFailedVolumes() {
+      return failedVolumes_;
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      capacity_ = 0L;
+      dfsUsed_ = 0L;
+      remaining_ = 0L;
+      blockPoolUsed_ = 0L;
+      xmitsInProgress_ = 0;
+      xceiverCount_ = 0;
+      failedVolumes_ = 0;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasRegistration()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasCapacity()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasDfsUsed()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasRemaining()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasBlockPoolUsed()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasXmitsInProgress()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasXceiverCount()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasFailedVolumes()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegistration().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, registration_);
+      }
+      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.writeUInt32(6, xmitsInProgress_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeUInt32(7, xceiverCount_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeUInt32(8, failedVolumes_);
+      }
+      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, registration_);
+      }
+      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
+          .computeUInt32Size(6, xmitsInProgress_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(7, xceiverCount_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(8, failedVolumes_);
+      }
+      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.DatanodeProtocolProtos.HeartbeatRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasRegistration() == other.hasRegistration());
+      if (hasRegistration()) {
+        result = result && getRegistration()
+            .equals(other.getRegistration());
+      }
+      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 && (hasXmitsInProgress() == other.hasXmitsInProgress());
+      if (hasXmitsInProgress()) {
+        result = result && (getXmitsInProgress()
+            == other.getXmitsInProgress());
+      }
+      result = result && (hasXceiverCount() == other.hasXceiverCount());
+      if (hasXceiverCount()) {
+        result = result && (getXceiverCount()
+            == other.getXceiverCount());
+      }
+      result = result && (hasFailedVolumes() == other.hasFailedVolumes());
+      if (hasFailedVolumes()) {
+        result = result && (getFailedVolumes()
+            == other.getFailedVolumes());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasRegistration()) {
+        hash = (37 * hash) + REGISTRATION_FIELD_NUMBER;
+        hash = (53 * hash) + getRegistration().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 (hasXmitsInProgress()) {
+        hash = (37 * hash) + XMITSINPROGRESS_FIELD_NUMBER;
+        hash = (53 * hash) + getXmitsInProgress();
+      }
+      if (hasXceiverCount()) {
+        hash = (37 * hash) + XCEIVERCOUNT_FIELD_NUMBER;
+        hash = (53 * hash) + getXceiverCount();
+      }
+      if (hasFailedVolumes()) {
+        hash = (37 * hash) + FAILEDVOLUMES_FIELD_NUMBER;
+        hash = (53 * hash) + getFailedVolumes();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto 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.DatanodeProtocolProtos.HeartbeatRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto 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.DatanodeProtocolProtos.HeartbeatRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto 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.DatanodeProtocolProtos.HeartbeatRequestProto 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.DatanodeProtocolProtos.HeartbeatRequestProto 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.DatanodeProtocolProtos.HeartbeatRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto 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.DatanodeProtocolProtos.HeartbeatRequestProto 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.DatanodeProtocolProtos.HeartbeatRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_HeartbeatRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_HeartbeatRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getRegistrationFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        capacity_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        dfsUsed_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        remaining_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        blockPoolUsed_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        xmitsInProgress_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        xceiverCount_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        failedVolumes_ = 0;
+        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.DatanodeProtocolProtos.HeartbeatRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (registrationBuilder_ == null) {
+          result.registration_ = registration_;
+        } else {
+          result.registration_ = registrationBuilder_.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.xmitsInProgress_ = xmitsInProgress_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.xceiverCount_ = xceiverCount_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.failedVolumes_ = failedVolumes_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto.getDefaultInstance()) return this;
+        if (other.hasRegistration()) {
+          mergeRegistration(other.getRegistration());
+        }
+        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.hasXmitsInProgress()) {
+          setXmitsInProgress(other.getXmitsInProgress());
+        }
+        if (other.hasXceiverCount()) {
+          setXceiverCount(other.getXceiverCount());
+        }
+        if (other.hasFailedVolumes()) {
+          setFailedVolumes(other.getFailedVolumes());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasRegistration()) {
+          
+          return false;
+        }
+        if (!hasCapacity()) {
+          
+          return false;
+        }
+        if (!hasDfsUsed()) {
+          
+          return false;
+        }
+        if (!hasRemaining()) {
+          
+          return false;
+        }
+        if (!hasBlockPoolUsed()) {
+          
+          return false;
+        }
+        if (!hasXmitsInProgress()) {
+          
+          return false;
+        }
+        if (!hasXceiverCount()) {
+          
+          return false;
+        }
+        if (!hasFailedVolumes()) {
+          
+          return false;
+        }
+        if (!getRegistration().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.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(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;
+              xmitsInProgress_ = input.readUInt32();
+              break;
+            }
+            case 56: {
+              bitField0_ |= 0x00000040;
+              xceiverCount_ = input.readUInt32();
+              break;
+            }
+            case 64: {
+              bitField0_ |= 0x00000080;
+              failedVolumes_ = input.readUInt32();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          registration_ = value;
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setRegistration(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder builderForValue) {
+        if (registrationBuilder_ == null) {
+          registration_ = builderForValue.build();
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder(registration_).mergeFrom(value).buildPartial();
+          } else {
+            registration_ = value;
+          }
+          onChanged();
+        } else {
+          registrationBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearRegistration() {
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // required 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;
+      }
+      
+      // required 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;
+      }
+      
+      // required 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;
+      }
+      
+      // required 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;
+      }
+      
+      // required uint32 xmitsInProgress = 6;
+      private int xmitsInProgress_ ;
+      public boolean hasXmitsInProgress() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      public int getXmitsInProgress() {
+        return xmitsInProgress_;
+      }
+      public Builder setXmitsInProgress(int value) {
+        bitField0_ |= 0x00000020;
+        xmitsInProgress_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearXmitsInProgress() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        xmitsInProgress_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // required 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;
+      }
+      
+      // required uint32 failedVolumes = 8;
+      private int failedVolumes_ ;
+      public boolean hasFailedVolumes() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      public int getFailedVolumes() {
+        return failedVolumes_;
+      }
+      public Builder setFailedVolumes(int value) {
+        bitField0_ |= 0x00000080;
+        failedVolumes_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearFailedVolumes() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        failedVolumes_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:HeartbeatRequestProto)
+    }
+    
+    static {
+      defaultInstance = new HeartbeatRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:HeartbeatRequestProto)
+  }
+  
+  public interface HeartbeatResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // repeated .DatanodeCommandProto cmds = 1;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto> 
+        getCmdsList();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto getCmds(int index);
+    int getCmdsCount();
+    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder> 
+        getCmdsOrBuilderList();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder getCmdsOrBuilder(
+        int index);
+  }
+  public static final class HeartbeatResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements HeartbeatResponseProtoOrBuilder {
+    // Use HeartbeatResponseProto.newBuilder() to construct.
+    private HeartbeatResponseProto(Builder builder) {
+      super(builder);
+    }
+    private HeartbeatResponseProto(boolean noInit) {}
+    
+    private static final HeartbeatResponseProto defaultInstance;
+    public static HeartbeatResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public HeartbeatResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_HeartbeatResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_HeartbeatResponseProto_fieldAccessorTable;
+    }
+    
+    // repeated .DatanodeCommandProto cmds = 1;
+    public static final int CMDS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto> cmds_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto> getCmdsList() {
+      return cmds_;
+    }
+    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder> 
+        getCmdsOrBuilderList() {
+      return cmds_;
+    }
+    public int getCmdsCount() {
+      return cmds_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto getCmds(int index) {
+      return cmds_.get(index);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder getCmdsOrBuilder(
+        int index) {
+      return cmds_.get(index);
+    }
+    
+    private void initFields() {
+      cmds_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      for (int i = 0; i < getCmdsCount(); i++) {
+        if (!getCmds(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < cmds_.size(); i++) {
+        output.writeMessage(1, cmds_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      for (int i = 0; i < cmds_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, cmds_.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.DatanodeProtocolProtos.HeartbeatResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto) obj;
+      
+      boolean result = true;
+      result = result && getCmdsList()
+          .equals(other.getCmdsList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (getCmdsCount() > 0) {
+        hash = (37 * hash) + CMDS_FIELD_NUMBER;
+        hash = (53 * hash) + getCmdsList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto 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.DatanodeProtocolProtos.HeartbeatResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto 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.DatanodeProtocolProtos.HeartbeatResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto 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.DatanodeProtocolProtos.HeartbeatResponseProto 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.DatanodeProtocolProtos.HeartbeatResponseProto 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.DatanodeProtocolProtos.HeartbeatResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto 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.DatanodeProtocolProtos.HeartbeatResponseProto 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.DatanodeProtocolProtos.HeartbeatResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_HeartbeatResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_HeartbeatResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getCmdsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (cmdsBuilder_ == null) {
+          cmds_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          cmdsBuilder_.clear();
+        }
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        if (cmdsBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            cmds_ = java.util.Collections.unmodifiableList(cmds_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.cmds_ = cmds_;
+        } else {
+          result.cmds_ = cmdsBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.getDefaultInstance()) return this;
+        if (cmdsBuilder_ == null) {
+          if (!other.cmds_.isEmpty()) {
+            if (cmds_.isEmpty()) {
+              cmds_ = other.cmds_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureCmdsIsMutable();
+              cmds_.addAll(other.cmds_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.cmds_.isEmpty()) {
+            if (cmdsBuilder_.isEmpty()) {
+              cmdsBuilder_.dispose();
+              cmdsBuilder_ = null;
+              cmds_ = other.cmds_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              cmdsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getCmdsFieldBuilder() : null;
+            } else {
+              cmdsBuilder_.addAllMessages(other.cmds_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        for (int i = 0; i < getCmdsCount(); i++) {
+          if (!getCmds(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.DatanodeProtocolProtos.DatanodeCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.newBuilder();
+              input.readMessage(subBuilder, extensionRegistry);
+              addCmds(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // repeated .DatanodeCommandProto cmds = 1;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto> cmds_ =
+        java.util.Collections.emptyList();
+      private void ensureCmdsIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          cmds_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto>(cmds_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+      
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder> cmdsBuilder_;
+      
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto> getCmdsList() {
+        if (cmdsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(cmds_);
+        } else {
+          return cmdsBuilder_.getMessageList();
+        }
+      }
+      public int getCmdsCount() {
+        if (cmdsBuilder_ == null) {
+          return cmds_.size();
+        } else {
+          return cmdsBuilder_.getCount();
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto getCmds(int index) {
+        if (cmdsBuilder_ == null) {
+          return cmds_.get(index);
+        } else {
+          return cmdsBuilder_.getMessage(index);
+        }
+      }
+      public Builder setCmds(
+          int index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto value) {
+        if (cmdsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureCmdsIsMutable();
+          cmds_.set(index, value);
+          onChanged();
+        } else {
+          cmdsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      public Builder setCmds(
+          int index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder builderForValue) {
+        if (cmdsBuilder_ == null) {
+          ensureCmdsIsMutable();
+          cmds_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          cmdsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addCmds(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto value) {
+        if (cmdsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureCmdsIsMutable();
+          cmds_.add(value);
+          onChanged();
+        } else {
+          cmdsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      public Builder addCmds(
+          int index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto value) {
+        if (cmdsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureCmdsIsMutable();
+          cmds_.add(index, value);
+          onChanged();
+        } else {
+          cmdsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      public Builder addCmds(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder builderForValue) {
+        if (cmdsBuilder_ == null) {
+          ensureCmdsIsMutable();
+          cmds_.add(builderForValue.build());
+          onChanged();
+        } else {
+          cmdsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addCmds(
+          int index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder builderForValue) {
+        if (cmdsBuilder_ == null) {
+          ensureCmdsIsMutable();
+          cmds_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          cmdsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addAllCmds(
+          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto> values) {
+        if (cmdsBuilder_ == null) {
+          ensureCmdsIsMutable();
+          super.addAll(values, cmds_);
+          onChanged();
+        } else {
+          cmdsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      public Builder clearCmds() {
+        if (cmdsBuilder_ == null) {
+          cmds_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          cmdsBuilder_.clear();
+        }
+        return this;
+      }
+      public Builder removeCmds(int index) {
+        if (cmdsBuilder_ == null) {
+          ensureCmdsIsMutable();
+          cmds_.remove(index);
+          onChanged();
+        } else {
+          cmdsBuilder_.remove(index);
+        }
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder getCmdsBuilder(
+          int index) {
+        return getCmdsFieldBuilder().getBuilder(index);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder getCmdsOrBuilder(
+          int index) {
+        if (cmdsBuilder_ == null) {
+          return cmds_.get(index);  } else {
+          return cmdsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder> 
+           getCmdsOrBuilderList() {
+        if (cmdsBuilder_ != null) {
+          return cmdsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(cmds_);
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder addCmdsBuilder() {
+        return getCmdsFieldBuilder().addBuilder(
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.getDefaultInstance());
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder addCmdsBuilder(
+          int index) {
+        return getCmdsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.getDefaultInstance());
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder> 
+           getCmdsBuilderList() {
+        return getCmdsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder> 
+          getCmdsFieldBuilder() {
+        if (cmdsBuilder_ == null) {
+          cmdsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder>(
+                  cmds_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          cmds_ = null;
+        }
+        return cmdsBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:HeartbeatResponseProto)
+    }
+    
+    static {
+      defaultInstance = new HeartbeatResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:HeartbeatResponseProto)
+  }
+  
+  public interface BlockReportRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+    
+    // required string blockPoolId = 2;
+    boolean hasBlockPoolId();
+    String getBlockPoolId();
+    
+    // repeated uint64 blocks = 3 [packed = true];
+    java.util.List<java.lang.Long> getBlocksList();
+    int getBlocksCount();
+    long getBlocks(int index);
+  }
+  public static final class BlockReportRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BlockReportRequestProtoOrBuilder {
+    // Use BlockReportRequestProto.newBuilder() to construct.
+    private BlockReportRequestProto(Builder builder) {
+      super(builder);
+    }
+    private BlockReportRequestProto(boolean noInit) {}
+    
+    private static final BlockReportRequestProto defaultInstance;
+    public static BlockReportRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BlockReportRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReportRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReportRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .DatanodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    // required string blockPoolId = 2;
+    public static final int BLOCKPOOLID_FIELD_NUMBER = 2;
+    private java.lang.Object blockPoolId_;
+    public boolean hasBlockPoolId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public String getBlockPoolId() {
+      java.lang.Object ref = blockPoolId_;
+      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)) {
+          blockPoolId_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getBlockPoolIdBytes() {
+      java.lang.Object ref = blockPoolId_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        blockPoolId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // repeated uint64 blocks = 3 [packed = true];
+    public static final int BLOCKS_FIELD_NUMBER = 3;
+    private java.util.List<java.lang.Long> blocks_;
+    public java.util.List<java.lang.Long>
+        getBlocksList() {
+      return blocks_;
+    }
+    public int getBlocksCount() {
+      return blocks_.size();
+    }
+    public long getBlocks(int index) {
+      return blocks_.get(index);
+    }
+    private int blocksMemoizedSerializedSize = -1;
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      blockPoolId_ = "";
+      blocks_ = java.util.Collections.emptyList();;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasRegistration()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasBlockPoolId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegistration().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, registration_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getBlockPoolIdBytes());
+      }
+      if (getBlocksList().size() > 0) {
+        output.writeRawVarint32(26);
+        output.writeRawVarint32(blocksMemoizedSerializedSize);
+      }
+      for (int i = 0; i < blocks_.size(); i++) {
+        output.writeUInt64NoTag(blocks_.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, registration_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getBlockPoolIdBytes());
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < blocks_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeUInt64SizeNoTag(blocks_.get(i));
+        }
+        size += dataSize;
+        if (!getBlocksList().isEmpty()) {
+          size += 1;
+          size += com.google.protobuf.CodedOutputStream
+              .computeInt32SizeNoTag(dataSize);
+        }
+        blocksMemoizedSerializedSize = dataSize;
+      }
+      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.DatanodeProtocolProtos.BlockReportRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasRegistration() == other.hasRegistration());
+      if (hasRegistration()) {
+        result = result && getRegistration()
+            .equals(other.getRegistration());
+      }
+      result = result && (hasBlockPoolId() == other.hasBlockPoolId());
+      if (hasBlockPoolId()) {
+        result = result && getBlockPoolId()
+            .equals(other.getBlockPoolId());
+      }
+      result = result && getBlocksList()
+          .equals(other.getBlocksList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasRegistration()) {
+        hash = (37 * hash) + REGISTRATION_FIELD_NUMBER;
+        hash = (53 * hash) + getRegistration().hashCode();
+      }
+      if (hasBlockPoolId()) {
+        hash = (37 * hash) + BLOCKPOOLID_FIELD_NUMBER;
+        hash = (53 * hash) + getBlockPoolId().hashCode();
+      }
+      if (getBlocksCount() > 0) {
+        hash = (37 * hash) + BLOCKS_FIELD_NUMBER;
+        hash = (53 * hash) + getBlocksList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto 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.DatanodeProtocolProtos.BlockReportRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto 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.DatanodeProtocolProtos.BlockReportRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto 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.DatanodeProtocolProtos.BlockReportRequestProto 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.DatanodeProtocolProtos.BlockReportRequestProto 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.DatanodeProtocolProtos.BlockReportRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto 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.DatanodeProtocolProtos.BlockReportRequestProto 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.DatanodeProtocolProtos.BlockReportRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReportRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReportRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getRegistrationFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        blockPoolId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        blocks_ = java.util.Collections.emptyList();;
+        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.DatanodeProtocolProtos.BlockReportRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (registrationBuilder_ == null) {
+          result.registration_ = registration_;
+        } else {
+          result.registration_ = registrationBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.blockPoolId_ = blockPoolId_;
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          blocks_ = java.util.Collections.unmodifiableList(blocks_);
+          bitField0_ = (bitField0_ & ~0x00000004);
+        }
+        result.blocks_ = blocks_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto.getDefaultInstance()) return this;
+        if (other.hasRegistration()) {
+          mergeRegistration(other.getRegistration());
+        }
+        if (other.hasBlockPoolId()) {
+          setBlockPoolId(other.getBlockPoolId());
+        }
+        if (!other.blocks_.isEmpty()) {
+          if (blocks_.isEmpty()) {
+            blocks_ = other.blocks_;
+            bitField0_ = (bitField0_ & ~0x00000004);
+          } else {
+            ensureBlocksIsMutable();
+            blocks_.addAll(other.blocks_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasRegistration()) {
+          
+          return false;
+        }
+        if (!hasBlockPoolId()) {
+          
+          return false;
+        }
+        if (!getRegistration().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.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              blockPoolId_ = input.readBytes();
+              break;
+            }
+            case 24: {
+              ensureBlocksIsMutable();
+              blocks_.add(input.readUInt64());
+              break;
+            }
+            case 26: {
+              int length = input.readRawVarint32();
+              int limit = input.pushLimit(length);
+              while (input.getBytesUntilLimit() > 0) {
+                addBlocks(input.readUInt64());
+              }
+              input.popLimit(limit);
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          registration_ = value;
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setRegistration(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder builderForValue) {
+        if (registrationBuilder_ == null) {
+          registration_ = builderForValue.build();
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder(registration_).mergeFrom(value).buildPartial();
+          } else {
+            registration_ = value;
+          }
+          onChanged();
+        } else {
+          registrationBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearRegistration() {
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // required string blockPoolId = 2;
+      private java.lang.Object blockPoolId_ = "";
+      public boolean hasBlockPoolId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public String getBlockPoolId() {
+        java.lang.Object ref = blockPoolId_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          blockPoolId_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setBlockPoolId(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        blockPoolId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearBlockPoolId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        blockPoolId_ = getDefaultInstance().getBlockPoolId();
+        onChanged();
+        return this;
+      }
+      void setBlockPoolId(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000002;
+        blockPoolId_ = value;
+        onChanged();
+      }
+      
+      // repeated uint64 blocks = 3 [packed = true];
+      private java.util.List<java.lang.Long> blocks_ = java.util.Collections.emptyList();;
+      private void ensureBlocksIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          blocks_ = new java.util.ArrayList<java.lang.Long>(blocks_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+      public java.util.List<java.lang.Long>
+          getBlocksList() {
+        return java.util.Collections.unmodifiableList(blocks_);
+      }
+      public int getBlocksCount() {
+        return blocks_.size();
+      }
+      public long getBlocks(int index) {
+        return blocks_.get(index);
+      }
+      public Builder setBlocks(
+          int index, long value) {
+        ensureBlocksIsMutable();
+        blocks_.set(index, value);
+        onChanged();
+        return this;
+      }
+      public Builder addBlocks(long value) {
+        ensureBlocksIsMutable();
+        blocks_.add(value);
+        onChanged();
+        return this;
+      }
+      public Builder addAllBlocks(
+          java.lang.Iterable<? extends java.lang.Long> values) {
+        ensureBlocksIsMutable();
+        super.addAll(values, blocks_);
+        onChanged();
+        return this;
+      }
+      public Builder clearBlocks() {
+        blocks_ = java.util.Collections.emptyList();;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:BlockReportRequestProto)
+    }
+    
+    static {
+      defaultInstance = new BlockReportRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BlockReportRequestProto)
+  }
+  
+  public interface BlockReportResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeCommandProto cmd = 1;
+    boolean hasCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto getCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder getCmdOrBuilder();
+  }
+  public static final class BlockReportResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BlockReportResponseProtoOrBuilder {
+    // Use BlockReportResponseProto.newBuilder() to construct.
+    private BlockReportResponseProto(Builder builder) {
+      super(builder);
+    }
+    private BlockReportResponseProto(boolean noInit) {}
+    
+    private static final BlockReportResponseProto defaultInstance;
+    public static BlockReportResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BlockReportResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReportResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReportResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .DatanodeCommandProto cmd = 1;
+    public static final int CMD_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto cmd_;
+    public boolean hasCmd() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto getCmd() {
+      return cmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder getCmdOrBuilder() {
+      return cmd_;
+    }
+    
+    private void initFields() {
+      cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasCmd()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getCmd().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, cmd_);
+      }
+      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, cmd_);
+      }
+      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.DatanodeProtocolProtos.BlockReportResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasCmd() == other.hasCmd());
+      if (hasCmd()) {
+        result = result && getCmd()
+            .equals(other.getCmd());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasCmd()) {
+        hash = (37 * hash) + CMD_FIELD_NUMBER;
+        hash = (53 * hash) + getCmd().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto 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.DatanodeProtocolProtos.BlockReportResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto 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.DatanodeProtocolProtos.BlockReportResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto 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.DatanodeProtocolProtos.BlockReportResponseProto 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.DatanodeProtocolProtos.BlockReportResponseProto 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.DatanodeProtocolProtos.BlockReportResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto 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.DatanodeProtocolProtos.BlockReportResponseProto 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.DatanodeProtocolProtos.BlockReportResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReportResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReportResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getCmdFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (cmdBuilder_ == null) {
+          cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.getDefaultInstance();
+        } else {
+          cmdBuilder_.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.DatanodeProtocolProtos.BlockReportResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (cmdBuilder_ == null) {
+          result.cmd_ = cmd_;
+        } else {
+          result.cmd_ = cmdBuilder_.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.DatanodeProtocolProtos.BlockReportResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.getDefaultInstance()) return this;
+        if (other.hasCmd()) {
+          mergeCmd(other.getCmd());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasCmd()) {
+          
+          return false;
+        }
+        if (!getCmd().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.DatanodeProtocolProtos.DatanodeCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.newBuilder();
+              if (hasCmd()) {
+                subBuilder.mergeFrom(getCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setCmd(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeCommandProto cmd = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder> cmdBuilder_;
+      public boolean hasCmd() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto getCmd() {
+        if (cmdBuilder_ == null) {
+          return cmd_;
+        } else {
+          return cmdBuilder_.getMessage();
+        }
+      }
+      public Builder setCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto value) {
+        if (cmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          cmd_ = value;
+          onChanged();
+        } else {
+          cmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder builderForValue) {
+        if (cmdBuilder_ == null) {
+          cmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          cmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto value) {
+        if (cmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              cmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.getDefaultInstance()) {
+            cmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.newBuilder(cmd_).mergeFrom(value).buildPartial();
+          } else {
+            cmd_ = value;
+          }
+          onChanged();
+        } else {
+          cmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearCmd() {
+        if (cmdBuilder_ == null) {
+          cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          cmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder getCmdBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder getCmdOrBuilder() {
+        if (cmdBuilder_ != null) {
+          return cmdBuilder_.getMessageOrBuilder();
+        } else {
+          return cmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder> 
+          getCmdFieldBuilder() {
+        if (cmdBuilder_ == null) {
+          cmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProtoOrBuilder>(
+                  cmd_,
+                  getParentForChildren(),
+                  isClean());
+          cmd_ = null;
+        }
+        return cmdBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:BlockReportResponseProto)
+    }
+    
+    static {
+      defaultInstance = new BlockReportResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BlockReportResponseProto)
+  }
+  
+  public interface ReceivedDeletedBlockInfoProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .BlockProto block = 1;
+    boolean hasBlock();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto getBlock();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder getBlockOrBuilder();
+    
+    // optional string deleteHint = 2;
+    boolean hasDeleteHint();
+    String getDeleteHint();
+  }
+  public static final class ReceivedDeletedBlockInfoProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ReceivedDeletedBlockInfoProtoOrBuilder {
+    // Use ReceivedDeletedBlockInfoProto.newBuilder() to construct.
+    private ReceivedDeletedBlockInfoProto(Builder builder) {
+      super(builder);
+    }
+    private ReceivedDeletedBlockInfoProto(boolean noInit) {}
+    
+    private static final ReceivedDeletedBlockInfoProto defaultInstance;
+    public static ReceivedDeletedBlockInfoProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ReceivedDeletedBlockInfoProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReceivedDeletedBlockInfoProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReceivedDeletedBlockInfoProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .BlockProto block = 1;
+    public static final int BLOCK_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto block_;
+    public boolean hasBlock() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto getBlock() {
+      return block_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder getBlockOrBuilder() {
+      return block_;
+    }
+    
+    // optional string deleteHint = 2;
+    public static final int DELETEHINT_FIELD_NUMBER = 2;
+    private java.lang.Object deleteHint_;
+    public boolean hasDeleteHint() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public String getDeleteHint() {
+      java.lang.Object ref = deleteHint_;
+      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)) {
+          deleteHint_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getDeleteHintBytes() {
+      java.lang.Object ref = deleteHint_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        deleteHint_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    private void initFields() {
+      block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance();
+      deleteHint_ = "";
+    }
+    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;
+      }
+      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.writeBytes(2, getDeleteHintBytes());
+      }
+      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
+          .computeBytesSize(2, getDeleteHintBytes());
+      }
+      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.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlock() == other.hasBlock());
+      if (hasBlock()) {
+        result = result && getBlock()
+            .equals(other.getBlock());
+      }
+      result = result && (hasDeleteHint() == other.hasDeleteHint());
+      if (hasDeleteHint()) {
+        result = result && getDeleteHint()
+            .equals(other.getDeleteHint());
+      }
+      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 (hasDeleteHint()) {
+        hash = (37 * hash) + DELETEHINT_FIELD_NUMBER;
+        hash = (53 * hash) + getDeleteHint().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto 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.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto 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.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto 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.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto 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.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto 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.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto 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.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto 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.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReceivedDeletedBlockInfoProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReceivedDeletedBlockInfoProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBlockFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (blockBuilder_ == null) {
+          block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        deleteHint_ = "";
+        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.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto(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;
+        }
+        result.deleteHint_ = deleteHint_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.getDefaultInstance()) return this;
+        if (other.hasBlock()) {
+          mergeBlock(other.getBlock());
+        }
+        if (other.hasDeleteHint()) {
+          setDeleteHint(other.getDeleteHint());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBlock()) {
+          
+          return false;
+        }
+        if (!getBlock().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.BlockProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.newBuilder();
+              if (hasBlock()) {
+                subBuilder.mergeFrom(getBlock());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setBlock(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              deleteHint_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .BlockProto block = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder> blockBuilder_;
+      public boolean hasBlock() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto getBlock() {
+        if (blockBuilder_ == null) {
+          return block_;
+        } else {
+          return blockBuilder_.getMessage();
+        }
+      }
+      public Builder setBlock(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto 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.BlockProto.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.BlockProto value) {
+        if (blockBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              block_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance()) {
+            block_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.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.BlockProto.getDefaultInstance();
+          onChanged();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder getBlockBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getBlockFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder getBlockOrBuilder() {
+        if (blockBuilder_ != null) {
+          return blockBuilder_.getMessageOrBuilder();
+        } else {
+          return block_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder> 
+          getBlockFieldBuilder() {
+        if (blockBuilder_ == null) {
+          blockBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder>(
+                  block_,
+                  getParentForChildren(),
+                  isClean());
+          block_ = null;
+        }
+        return blockBuilder_;
+      }
+      
+      // optional string deleteHint = 2;
+      private java.lang.Object deleteHint_ = "";
+      public boolean hasDeleteHint() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public String getDeleteHint() {
+        java.lang.Object ref = deleteHint_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          deleteHint_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setDeleteHint(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        deleteHint_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearDeleteHint() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        deleteHint_ = getDefaultInstance().getDeleteHint();
+        onChanged();
+        return this;
+      }
+      void setDeleteHint(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000002;
+        deleteHint_ = value;
+        onChanged();
+      }
+      
+      // @@protoc_insertion_point(builder_scope:ReceivedDeletedBlockInfoProto)
+    }
+    
+    static {
+      defaultInstance = new ReceivedDeletedBlockInfoProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ReceivedDeletedBlockInfoProto)
+  }
+  
+  public interface BlockReceivedAndDeletedRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+    
+    // required string blockPoolId = 2;
+    boolean hasBlockPoolId();
+    String getBlockPoolId();
+    
+    // repeated .ReceivedDeletedBlockInfoProto blocks = 3;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto> 
+        getBlocksList();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto getBlocks(int index);
+    int getBlocksCount();
+    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder> 
+        getBlocksOrBuilderList();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder getBlocksOrBuilder(
+        int index);
+  }
+  public static final class BlockReceivedAndDeletedRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BlockReceivedAndDeletedRequestProtoOrBuilder {
+    // Use BlockReceivedAndDeletedRequestProto.newBuilder() to construct.
+    private BlockReceivedAndDeletedRequestProto(Builder builder) {
+      super(builder);
+    }
+    private BlockReceivedAndDeletedRequestProto(boolean noInit) {}
+    
+    private static final BlockReceivedAndDeletedRequestProto defaultInstance;
+    public static BlockReceivedAndDeletedRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BlockReceivedAndDeletedRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReceivedAndDeletedRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReceivedAndDeletedRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .DatanodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    // required string blockPoolId = 2;
+    public static final int BLOCKPOOLID_FIELD_NUMBER = 2;
+    private java.lang.Object blockPoolId_;
+    public boolean hasBlockPoolId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public String getBlockPoolId() {
+      java.lang.Object ref = blockPoolId_;
+      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)) {
+          blockPoolId_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getBlockPoolIdBytes() {
+      java.lang.Object ref = blockPoolId_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        blockPoolId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // repeated .ReceivedDeletedBlockInfoProto blocks = 3;
+    public static final int BLOCKS_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto> blocks_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto> getBlocksList() {
+      return blocks_;
+    }
+    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder> 
+        getBlocksOrBuilderList() {
+      return blocks_;
+    }
+    public int getBlocksCount() {
+      return blocks_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto getBlocks(int index) {
+      return blocks_.get(index);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder getBlocksOrBuilder(
+        int index) {
+      return blocks_.get(index);
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      blockPoolId_ = "";
+      blocks_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasRegistration()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasBlockPoolId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegistration().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getBlocksCount(); i++) {
+        if (!getBlocks(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, registration_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getBlockPoolIdBytes());
+      }
+      for (int i = 0; i < blocks_.size(); i++) {
+        output.writeMessage(3, blocks_.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, registration_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getBlockPoolIdBytes());
+      }
+      for (int i = 0; i < blocks_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, blocks_.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.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasRegistration() == other.hasRegistration());
+      if (hasRegistration()) {
+        result = result && getRegistration()
+            .equals(other.getRegistration());
+      }
+      result = result && (hasBlockPoolId() == other.hasBlockPoolId());
+      if (hasBlockPoolId()) {
+        result = result && getBlockPoolId()
+            .equals(other.getBlockPoolId());
+      }
+      result = result && getBlocksList()
+          .equals(other.getBlocksList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasRegistration()) {
+        hash = (37 * hash) + REGISTRATION_FIELD_NUMBER;
+        hash = (53 * hash) + getRegistration().hashCode();
+      }
+      if (hasBlockPoolId()) {
+        hash = (37 * hash) + BLOCKPOOLID_FIELD_NUMBER;
+        hash = (53 * hash) + getBlockPoolId().hashCode();
+      }
+      if (getBlocksCount() > 0) {
+        hash = (37 * hash) + BLOCKS_FIELD_NUMBER;
+        hash = (53 * hash) + getBlocksList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReceivedAndDeletedRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReceivedAndDeletedRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getRegistrationFieldBuilder();
+          getBlocksFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        blockPoolId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (blocksBuilder_ == null) {
+          blocks_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        } else {
+          blocksBuilder_.clear();
+        }
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (registrationBuilder_ == null) {
+          result.registration_ = registration_;
+        } else {
+          result.registration_ = registrationBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.blockPoolId_ = blockPoolId_;
+        if (blocksBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            blocks_ = java.util.Collections.unmodifiableList(blocks_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.blocks_ = blocks_;
+        } else {
+          result.blocks_ = blocksBuilder_.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.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto.getDefaultInstance()) return this;
+        if (other.hasRegistration()) {
+          mergeRegistration(other.getRegistration());
+        }
+        if (other.hasBlockPoolId()) {
+          setBlockPoolId(other.getBlockPoolId());
+        }
+        if (blocksBuilder_ == null) {
+          if (!other.blocks_.isEmpty()) {
+            if (blocks_.isEmpty()) {
+              blocks_ = other.blocks_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+            } else {
+              ensureBlocksIsMutable();
+              blocks_.addAll(other.blocks_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.blocks_.isEmpty()) {
+            if (blocksBuilder_.isEmpty()) {
+              blocksBuilder_.dispose();
+              blocksBuilder_ = null;
+              blocks_ = other.blocks_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+              blocksBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getBlocksFieldBuilder() : null;
+            } else {
+              blocksBuilder_.addAllMessages(other.blocks_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasRegistration()) {
+          
+          return false;
+        }
+        if (!hasBlockPoolId()) {
+          
+          return false;
+        }
+        if (!getRegistration().isInitialized()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getBlocksCount(); i++) {
+          if (!getBlocks(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.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              blockPoolId_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.newBuilder();
+              input.readMessage(subBuilder, extensionRegistry);
+              addBlocks(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          registration_ = value;
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setRegistration(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder builderForValue) {
+        if (registrationBuilder_ == null) {
+          registration_ = builderForValue.build();
+          onChanged();
+        } else {
+          registrationBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeRegistration(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder(registration_).mergeFrom(value).buildPartial();
+          } else {
+            registration_ = value;
+          }
+          onChanged();
+        } else {
+          registrationBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearRegistration() {
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // required string blockPoolId = 2;
+      private java.lang.Object blockPoolId_ = "";
+      public boolean hasBlockPoolId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public String getBlockPoolId() {
+        java.lang.Object ref = blockPoolId_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          blockPoolId_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setBlockPoolId(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        blockPoolId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearBlockPoolId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        blockPoolId_ = getDefaultInstance().getBlockPoolId();
+        onChanged();
+        return this;
+      }
+      void setBlockPoolId(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000002;
+        blockPoolId_ = value;
+        onChanged();
+      }
+      
+      // repeated .ReceivedDeletedBlockInfoProto blocks = 3;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto> blocks_ =
+        java.util.Collections.emptyList();
+      private void ensureBlocksIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          blocks_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto>(blocks_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+      
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder> blocksBuilder_;
+      
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto> getBlocksList() {
+        if (blocksBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(blocks_);
+        } else {
+          return blocksBuilder_.getMessageList();
+        }
+      }
+      public int getBlocksCount() {
+        if (blocksBuilder_ == null) {
+          return blocks_.size();
+        } else {
+          return blocksBuilder_.getCount();
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto getBlocks(int index) {
+        if (blocksBuilder_ == null) {
+          return blocks_.get(index);
+        } else {
+          return blocksBuilder_.getMessage(index);
+        }
+      }
+      public Builder setBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.set(index, value);
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      public Builder setBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addBlocks(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.add(value);
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.add(index, value);
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.add(builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addAllBlocks(
+          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto> values) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          super.addAll(values, blocks_);
+          onChanged();
+        } else {
+          blocksBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      public Builder clearBlocks() {
+        if (blocksBuilder_ == null) {
+          blocks_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+          onChanged();
+        } else {
+          blocksBuilder_.clear();
+        }
+        return this;
+      }
+      public Builder removeBlocks(int index) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.remove(index);
+          onChanged();
+        } else {
+          blocksBuilder_.remove(index);
+        }
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder getBlocksBuilder(
+          int index) {
+        return getBlocksFieldBuilder().getBuilder(index);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder getBlocksOrBuilder(
+          int index) {
+        if (blocksBuilder_ == null) {
+          return blocks_.get(index);  } else {
+          return blocksBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder> 
+           getBlocksOrBuilderList() {
+        if (blocksBuilder_ != null) {
+          return blocksBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(blocks_);
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder addBlocksBuilder() {
+        return getBlocksFieldBuilder().addBuilder(
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.getDefaultInstance());
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder addBlocksBuilder(
+          int index) {
+        return getBlocksFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.getDefaultInstance());
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder> 
+           getBlocksBuilderList() {
+        return getBlocksFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder> 
+          getBlocksFieldBuilder() {
+        if (blocksBuilder_ == null) {
+          blocksBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProtoOrBuilder>(
+                  blocks_,
+                  ((bitField0_ & 0x00000004) == 0x00000004),
+                  getParentForChildren(),
+                  isClean());
+          blocks_ = null;
+        }
+        return blocksBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:BlockReceivedAndDeletedRequestProto)
+    }
+    
+    static {
+      defaultInstance = new BlockReceivedAndDeletedRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BlockReceivedAndDeletedRequestProto)
+  }
+  
+  public interface BlockReceivedAndDeletedResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class BlockReceivedAndDeletedResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BlockReceivedAndDeletedResponseProtoOrBuilder {
+    // Use BlockReceivedAndDeletedResponseProto.newBuilder() to construct.
+    private BlockReceivedAndDeletedResponseProto(Builder builder) {
+      super(builder);
+    }
+    private BlockReceivedAndDeletedResponseProto(boolean noInit) {}
+    
+    private static final BlockReceivedAndDeletedResponseProto defaultInstance;
+    public static BlockReceivedAndDeletedResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BlockReceivedAndDeletedResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReceivedAndDeletedResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReceivedAndDeletedResponseProto_fieldAccessorTable;
+    }
+    
+    private void initFields() {
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      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.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto) obj;
+      
+      boolean result = true;
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto 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.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReceivedAndDeletedResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_BlockReceivedAndDeletedResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.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();
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        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;
+            }
+          }
+        }
+      }
+      
+      
+      // @@protoc_insertion_point(builder_scope:BlockReceivedAndDeletedResponseProto)
+    }
+    
+    static {
+      defaultInstance = new BlockReceivedAndDeletedResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BlockReceivedAndDeletedResponseProto)
+  }
+  
+  public interface ErrorReportRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeRegistrationProto registartion = 1;
+    boolean hasRegistartion();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistartion();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistartionOrBuilder();
+    
+    // required uint32 errorCode = 2;
+    boolean hasErrorCode();
+    int getErrorCode();
+    
+    // required string msg = 3;
+    boolean hasMsg();
+    String getMsg();
+  }
+  public static final class ErrorReportRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ErrorReportRequestProtoOrBuilder {
+    // Use ErrorReportRequestProto.newBuilder() to construct.
+    private ErrorReportRequestProto(Builder builder) {
+      super(builder);
+    }
+    private ErrorReportRequestProto(boolean noInit) {}
+    
+    private static final ErrorReportRequestProto defaultInstance;
+    public static ErrorReportRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ErrorReportRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ErrorReportRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ErrorReportRequestProto_fieldAccessorTable;
+    }
+    
+    public enum ErrorCode
+        implements com.google.protobuf.ProtocolMessageEnum {
+      NOTIFY(0, 0),
+      DISK_ERROR(1, 1),
+      INVALID_BLOCK(2, 2),
+      FATAL_DISK_ERROR(3, 3),
+      ;
+      
+      public static final int NOTIFY_VALUE = 0;
+      public static final int DISK_ERROR_VALUE = 1;
+      public static final int INVALID_BLOCK_VALUE = 2;
+      public static final int FATAL_DISK_ERROR_VALUE = 3;
+      
+      
+      public final int getNumber() { return value; }
+      
+      public static ErrorCode valueOf(int value) {
+        switch (value) {
+          case 0: return NOTIFY;
+          case 1: return DISK_ERROR;
+          case 2: return INVALID_BLOCK;
+          case 3: return FATAL_DISK_ERROR;
+          default: return null;
+        }
+      }
+      
+      public static com.google.protobuf.Internal.EnumLiteMap<ErrorCode>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<ErrorCode>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<ErrorCode>() {
+              public ErrorCode findValueByNumber(int number) {
+                return ErrorCode.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.DatanodeProtocolProtos.ErrorReportRequestProto.getDescriptor().getEnumTypes().get(0);
+      }
+      
+      private static final ErrorCode[] VALUES = {
+        NOTIFY, DISK_ERROR, INVALID_BLOCK, FATAL_DISK_ERROR, 
+      };
+      
+      public static ErrorCode 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 ErrorCode(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+      
+      // @@protoc_insertion_point(enum_scope:ErrorReportRequestProto.ErrorCode)
+    }
+    
+    private int bitField0_;
+    // required .DatanodeRegistrationProto registartion = 1;
+    public static final int REGISTARTION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registartion_;
+    public boolean hasRegistartion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistartion() {
+      return registartion_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistartionOrBuilder() {
+      return registartion_;
+    }
+    
+    // required uint32 errorCode = 2;
+    public static final int ERRORCODE_FIELD_NUMBER = 2;
+    private int errorCode_;
+    public boolean hasErrorCode() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public int getErrorCode() {
+      return errorCode_;
+    }
+    
+    // required string msg = 3;
+    public static final int MSG_FIELD_NUMBER = 3;
+    private java.lang.Object msg_;
+    public boolean hasMsg() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public String getMsg() {
+      java.lang.Object ref = msg_;
+      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)) {
+          msg_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getMsgBytes() {
+      java.lang.Object ref = msg_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        msg_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    private void initFields() {
+      registartion_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      errorCode_ = 0;
+      msg_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasRegistartion()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasErrorCode()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasMsg()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegistartion().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, registartion_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt32(2, errorCode_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, getMsgBytes());
+      }
+      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, registartion_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(2, errorCode_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getMsgBytes());
+      }
+      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.DatanodeProtocolProtos.ErrorReportRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasRegistartion() == other.hasRegistartion());
+      if (hasRegistartion()) {
+        result = result && getRegistartion()
+            .equals(other.getRegistartion());
+      }
+      result = result && (hasErrorCode() == other.hasErrorCode());
+      if (hasErrorCode()) {
+        result = result && (getErrorCode()
+            == other.getErrorCode());
+      }
+      result = result && (hasMsg() == other.hasMsg());
+      if (hasMsg()) {
+        result = result && getMsg()
+            .equals(other.getMsg());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasRegistartion()) {
+        hash = (37 * hash) + REGISTARTION_FIELD_NUMBER;
+        hash = (53 * hash) + getRegistartion().hashCode();
+      }
+      if (hasErrorCode()) {
+        hash = (37 * hash) + ERRORCODE_FIELD_NUMBER;
+        hash = (53 * hash) + getErrorCode();
+      }
+      if (hasMsg()) {
+        hash = (37 * hash) + MSG_FIELD_NUMBER;
+        hash = (53 * hash) + getMsg().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto 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.DatanodeProtocolProtos.ErrorReportRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto 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.DatanodeProtocolProtos.ErrorReportRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto 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.DatanodeProtocolProtos.ErrorReportRequestProto 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.DatanodeProtocolProtos.ErrorReportRequestProto 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.DatanodeProtocolProtos.ErrorReportRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto 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.DatanodeProtocolProtos.ErrorReportRequestProto 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.DatanodeProtocolProtos.ErrorReportRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ErrorReportRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ErrorReportRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getRegistartionFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (registartionBuilder_ == null) {
+          registartion_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+        } else {
+          registartionBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        errorCode_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        msg_ = "";
+        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.DatanodeProtocolProtos.ErrorReportRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (registartionBuilder_ == null) {
+          result.registartion_ = registartion_;
+        } else {
+          result.registartion_ = registartionBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.errorCode_ = errorCode_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.msg_ = msg_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto.getDefaultInstance()) return this;
+        if (other.hasRegistartion()) {
+          mergeRegistartion(other.getRegistartion());
+        }
+        if (other.hasErrorCode()) {
+          setErrorCode(other.getErrorCode());
+        }
+        if (other.hasMsg()) {
+          setMsg(other.getMsg());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasRegistartion()) {
+          
+          return false;
+        }
+        if (!hasErrorCode()) {
+          
+          return false;
+        }
+        if (!hasMsg()) {
+          
+          return false;
+        }
+        if (!getRegistartion().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.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder();
+              if (hasRegistartion()) {
+                subBuilder.mergeFrom(getRegistartion());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistartion(subBuilder.buildPartial());
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              errorCode_ = input.readUInt32();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              msg_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeRegistrationProto registartion = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto registartion_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> registartionBuilder_;
+      public boolean hasRegistartion() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto getRegistartion() {
+        if (registartionBuilder_ == null) {
+          return registartion_;
+        } else {
+          return registartionBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistartion(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registartionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          registartion_ = value;
+          onChanged();
+        } else {
+          registartionBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setRegistartion(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder builderForValue) {
+        if (registartionBuilder_ == null) {
+          registartion_ = builderForValue.build();
+          onChanged();
+        } else {
+          registartionBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeRegistartion(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto value) {
+        if (registartionBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registartion_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance()) {
+            registartion_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.newBuilder(registartion_).mergeFrom(value).buildPartial();
+          } else {
+            registartion_ = value;
+          }
+          onChanged();
+        } else {
+          registartionBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearRegistartion() {
+        if (registartionBuilder_ == null) {
+          registartion_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registartionBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder getRegistartionBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistartionFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder getRegistartionOrBuilder() {
+        if (registartionBuilder_ != null) {
+          return registartionBuilder_.getMessageOrBuilder();
+        } else {
+          return registartion_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder> 
+          getRegistartionFieldBuilder() {
+        if (registartionBuilder_ == null) {
+          registartionBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProtoOrBuilder>(
+                  registartion_,
+                  getParentForChildren(),
+                  isClean());
+          registartion_ = null;
+        }
+        return registartionBuilder_;
+      }
+      
+      // required uint32 errorCode = 2;
+      private int errorCode_ ;
+      public boolean hasErrorCode() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public int getErrorCode() {
+        return errorCode_;
+      }
+      public Builder setErrorCode(int value) {
+        bitField0_ |= 0x00000002;
+        errorCode_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearErrorCode() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        errorCode_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // required string msg = 3;
+      private java.lang.Object msg_ = "";
+      public boolean hasMsg() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public String getMsg() {
+        java.lang.Object ref = msg_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          msg_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setMsg(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        msg_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearMsg() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        msg_ = getDefaultInstance().getMsg();
+        onChanged();
+        return this;
+      }
+      void setMsg(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000004;
+        msg_ = value;
+        onChanged();
+      }
+      
+      // @@protoc_insertion_point(builder_scope:ErrorReportRequestProto)
+    }
+    
+    static {
+      defaultInstance = new ErrorReportRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ErrorReportRequestProto)
+  }
+  
+  public interface ErrorReportResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class ErrorReportResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ErrorReportResponseProtoOrBuilder {
+    // Use ErrorReportResponseProto.newBuilder() to construct.
+    private ErrorReportResponseProto(Builder builder) {
+      super(builder);
+    }
+    private ErrorReportResponseProto(boolean noInit) {}
+    
+    private static final ErrorReportResponseProto defaultInstance;
+    public static ErrorReportResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ErrorReportResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ErrorReportResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ErrorReportResponseProto_fieldAccessorTable;
+    }
+    
+    private void initFields() {
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      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.DatanodeProtocolProtos.ErrorReportResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto) obj;
+      
+      boolean result = true;
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto 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.DatanodeProtocolProtos.ErrorReportResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto 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.DatanodeProtocolProtos.ErrorReportResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto 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.DatanodeProtocolProtos.ErrorReportResponseProto 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.DatanodeProtocolProtos.ErrorReportResponseProto 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.DatanodeProtocolProtos.ErrorReportResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto 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.DatanodeProtocolProtos.ErrorReportResponseProto 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.DatanodeProtocolProtos.ErrorReportResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ErrorReportResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ErrorReportResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.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();
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        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;
+            }
+          }
+        }
+      }
+      
+      
+      // @@protoc_insertion_point(builder_scope:ErrorReportResponseProto)
+    }
+    
+    static {
+      defaultInstance = new ErrorReportResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ErrorReportResponseProto)
+  }
+  
+  public interface ProcessUpgradeRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // optional .UpgradeCommandProto cmd = 1;
+    boolean hasCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder getCmdOrBuilder();
+  }
+  public static final class ProcessUpgradeRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ProcessUpgradeRequestProtoOrBuilder {
+    // Use ProcessUpgradeRequestProto.newBuilder() to construct.
+    private ProcessUpgradeRequestProto(Builder builder) {
+      super(builder);
+    }
+    private ProcessUpgradeRequestProto(boolean noInit) {}
+    
+    private static final ProcessUpgradeRequestProto defaultInstance;
+    public static ProcessUpgradeRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ProcessUpgradeRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ProcessUpgradeRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ProcessUpgradeRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // optional .UpgradeCommandProto cmd = 1;
+    public static final int CMD_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto cmd_;
+    public boolean hasCmd() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getCmd() {
+      return cmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder getCmdOrBuilder() {
+      return cmd_;
+    }
+    
+    private void initFields() {
+      cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (hasCmd()) {
+        if (!getCmd().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, cmd_);
+      }
+      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, cmd_);
+      }
+      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.DatanodeProtocolProtos.ProcessUpgradeRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasCmd() == other.hasCmd());
+      if (hasCmd()) {
+        result = result && getCmd()
+            .equals(other.getCmd());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasCmd()) {
+        hash = (37 * hash) + CMD_FIELD_NUMBER;
+        hash = (53 * hash) + getCmd().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto 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.DatanodeProtocolProtos.ProcessUpgradeRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto 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.DatanodeProtocolProtos.ProcessUpgradeRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto 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.DatanodeProtocolProtos.ProcessUpgradeRequestProto 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.DatanodeProtocolProtos.ProcessUpgradeRequestProto 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.DatanodeProtocolProtos.ProcessUpgradeRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto 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.DatanodeProtocolProtos.ProcessUpgradeRequestProto 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.DatanodeProtocolProtos.ProcessUpgradeRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ProcessUpgradeRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ProcessUpgradeRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getCmdFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (cmdBuilder_ == null) {
+          cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+        } else {
+          cmdBuilder_.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.DatanodeProtocolProtos.ProcessUpgradeRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (cmdBuilder_ == null) {
+          result.cmd_ = cmd_;
+        } else {
+          result.cmd_ = cmdBuilder_.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.DatanodeProtocolProtos.ProcessUpgradeRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto.getDefaultInstance()) return this;
+        if (other.hasCmd()) {
+          mergeCmd(other.getCmd());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (hasCmd()) {
+          if (!getCmd().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.DatanodeProtocolProtos.UpgradeCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.newBuilder();
+              if (hasCmd()) {
+                subBuilder.mergeFrom(getCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setCmd(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // optional .UpgradeCommandProto cmd = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder> cmdBuilder_;
+      public boolean hasCmd() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getCmd() {
+        if (cmdBuilder_ == null) {
+          return cmd_;
+        } else {
+          return cmdBuilder_.getMessage();
+        }
+      }
+      public Builder setCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto value) {
+        if (cmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          cmd_ = value;
+          onChanged();
+        } else {
+          cmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder builderForValue) {
+        if (cmdBuilder_ == null) {
+          cmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          cmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto value) {
+        if (cmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              cmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance()) {
+            cmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.newBuilder(cmd_).mergeFrom(value).buildPartial();
+          } else {
+            cmd_ = value;
+          }
+          onChanged();
+        } else {
+          cmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearCmd() {
+        if (cmdBuilder_ == null) {
+          cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          cmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder getCmdBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder getCmdOrBuilder() {
+        if (cmdBuilder_ != null) {
+          return cmdBuilder_.getMessageOrBuilder();
+        } else {
+          return cmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder> 
+          getCmdFieldBuilder() {
+        if (cmdBuilder_ == null) {
+          cmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder>(
+                  cmd_,
+                  getParentForChildren(),
+                  isClean());
+          cmd_ = null;
+        }
+        return cmdBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:ProcessUpgradeRequestProto)
+    }
+    
+    static {
+      defaultInstance = new ProcessUpgradeRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ProcessUpgradeRequestProto)
+  }
+  
+  public interface ProcessUpgradeResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // optional .UpgradeCommandProto cmd = 1;
+    boolean hasCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getCmd();
+    org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder getCmdOrBuilder();
+  }
+  public static final class ProcessUpgradeResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ProcessUpgradeResponseProtoOrBuilder {
+    // Use ProcessUpgradeResponseProto.newBuilder() to construct.
+    private ProcessUpgradeResponseProto(Builder builder) {
+      super(builder);
+    }
+    private ProcessUpgradeResponseProto(boolean noInit) {}
+    
+    private static final ProcessUpgradeResponseProto defaultInstance;
+    public static ProcessUpgradeResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ProcessUpgradeResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ProcessUpgradeResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ProcessUpgradeResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // optional .UpgradeCommandProto cmd = 1;
+    public static final int CMD_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto cmd_;
+    public boolean hasCmd() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getCmd() {
+      return cmd_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder getCmdOrBuilder() {
+      return cmd_;
+    }
+    
+    private void initFields() {
+      cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (hasCmd()) {
+        if (!getCmd().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, cmd_);
+      }
+      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, cmd_);
+      }
+      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.DatanodeProtocolProtos.ProcessUpgradeResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasCmd() == other.hasCmd());
+      if (hasCmd()) {
+        result = result && getCmd()
+            .equals(other.getCmd());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasCmd()) {
+        hash = (37 * hash) + CMD_FIELD_NUMBER;
+        hash = (53 * hash) + getCmd().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto 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.DatanodeProtocolProtos.ProcessUpgradeResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto 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.DatanodeProtocolProtos.ProcessUpgradeResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto 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.DatanodeProtocolProtos.ProcessUpgradeResponseProto 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.DatanodeProtocolProtos.ProcessUpgradeResponseProto 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.DatanodeProtocolProtos.ProcessUpgradeResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto 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.DatanodeProtocolProtos.ProcessUpgradeResponseProto 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.DatanodeProtocolProtos.ProcessUpgradeResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ProcessUpgradeResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ProcessUpgradeResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getCmdFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (cmdBuilder_ == null) {
+          cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+        } else {
+          cmdBuilder_.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.DatanodeProtocolProtos.ProcessUpgradeResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (cmdBuilder_ == null) {
+          result.cmd_ = cmd_;
+        } else {
+          result.cmd_ = cmdBuilder_.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.DatanodeProtocolProtos.ProcessUpgradeResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.getDefaultInstance()) return this;
+        if (other.hasCmd()) {
+          mergeCmd(other.getCmd());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (hasCmd()) {
+          if (!getCmd().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.DatanodeProtocolProtos.UpgradeCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.newBuilder();
+              if (hasCmd()) {
+                subBuilder.mergeFrom(getCmd());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setCmd(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // optional .UpgradeCommandProto cmd = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder> cmdBuilder_;
+      public boolean hasCmd() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto getCmd() {
+        if (cmdBuilder_ == null) {
+          return cmd_;
+        } else {
+          return cmdBuilder_.getMessage();
+        }
+      }
+      public Builder setCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto value) {
+        if (cmdBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          cmd_ = value;
+          onChanged();
+        } else {
+          cmdBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setCmd(
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder builderForValue) {
+        if (cmdBuilder_ == null) {
+          cmd_ = builderForValue.build();
+          onChanged();
+        } else {
+          cmdBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeCmd(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto value) {
+        if (cmdBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              cmd_ != org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance()) {
+            cmd_ =
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.newBuilder(cmd_).mergeFrom(value).buildPartial();
+          } else {
+            cmd_ = value;
+          }
+          onChanged();
+        } else {
+          cmdBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearCmd() {
+        if (cmdBuilder_ == null) {
+          cmd_ = org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          cmdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder getCmdBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getCmdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder getCmdOrBuilder() {
+        if (cmdBuilder_ != null) {
+          return cmdBuilder_.getMessageOrBuilder();
+        } else {
+          return cmd_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder> 
+          getCmdFieldBuilder() {
+        if (cmdBuilder_ == null) {
+          cmdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProtoOrBuilder>(
+                  cmd_,
+                  getParentForChildren(),
+                  isClean());
+          cmd_ = null;
+        }
+        return cmdBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:ProcessUpgradeResponseProto)
+    }
+    
+    static {
+      defaultInstance = new ProcessUpgradeResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ProcessUpgradeResponseProto)
+  }
+  
+  public interface ReportBadBlocksRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // repeated .LocatedBlockProto blocks = 1;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto> 
+        getBlocksList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto getBlocks(int index);
+    int getBlocksCount();
+    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProtoOrBuilder> 
+        getBlocksOrBuilderList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProtoOrBuilder getBlocksOrBuilder(
+        int index);
+  }
+  public static final class ReportBadBlocksRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ReportBadBlocksRequestProtoOrBuilder {
+    // Use ReportBadBlocksRequestProto.newBuilder() to construct.
+    private ReportBadBlocksRequestProto(Builder builder) {
+      super(builder);
+    }
+    private ReportBadBlocksRequestProto(boolean noInit) {}
+    
+    private static final ReportBadBlocksRequestProto defaultInstance;
+    public static ReportBadBlocksRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ReportBadBlocksRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReportBadBlocksRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReportBadBlocksRequestProto_fieldAccessorTable;
+    }
+    
+    // repeated .LocatedBlockProto blocks = 1;
+    public static final int BLOCKS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto> blocks_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto> getBlocksList() {
+      return blocks_;
+    }
+    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProtoOrBuilder> 
+        getBlocksOrBuilderList() {
+      return blocks_;
+    }
+    public int getBlocksCount() {
+      return blocks_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto getBlocks(int index) {
+      return blocks_.get(index);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProtoOrBuilder getBlocksOrBuilder(
+        int index) {
+      return blocks_.get(index);
+    }
+    
+    private void initFields() {
+      blocks_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      for (int i = 0; i < getBlocksCount(); i++) {
+        if (!getBlocks(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < blocks_.size(); i++) {
+        output.writeMessage(1, blocks_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      for (int i = 0; i < blocks_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, blocks_.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.DatanodeProtocolProtos.ReportBadBlocksRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto) obj;
+      
+      boolean result = true;
+      result = result && getBlocksList()
+          .equals(other.getBlocksList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (getBlocksCount() > 0) {
+        hash = (37 * hash) + BLOCKS_FIELD_NUMBER;
+        hash = (53 * hash) + getBlocksList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto 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.DatanodeProtocolProtos.ReportBadBlocksRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto 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.DatanodeProtocolProtos.ReportBadBlocksRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto 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.DatanodeProtocolProtos.ReportBadBlocksRequestProto 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.DatanodeProtocolProtos.ReportBadBlocksRequestProto 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.DatanodeProtocolProtos.ReportBadBlocksRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto 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.DatanodeProtocolProtos.ReportBadBlocksRequestProto 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.DatanodeProtocolProtos.ReportBadBlocksRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReportBadBlocksRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReportBadBlocksRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBlocksFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (blocksBuilder_ == null) {
+          blocks_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          blocksBuilder_.clear();
+        }
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        if (blocksBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            blocks_ = java.util.Collections.unmodifiableList(blocks_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.blocks_ = blocks_;
+        } else {
+          result.blocks_ = blocksBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto.getDefaultInstance()) return this;
+        if (blocksBuilder_ == null) {
+          if (!other.blocks_.isEmpty()) {
+            if (blocks_.isEmpty()) {
+              blocks_ = other.blocks_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureBlocksIsMutable();
+              blocks_.addAll(other.blocks_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.blocks_.isEmpty()) {
+            if (blocksBuilder_.isEmpty()) {
+              blocksBuilder_.dispose();
+              blocksBuilder_ = null;
+              blocks_ = other.blocks_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              blocksBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getBlocksFieldBuilder() : null;
+            } else {
+              blocksBuilder_.addAllMessages(other.blocks_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        for (int i = 0; i < getBlocksCount(); i++) {
+          if (!getBlocks(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.HdfsProtos.LocatedBlockProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.newBuilder();
+              input.readMessage(subBuilder, extensionRegistry);
+              addBlocks(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // repeated .LocatedBlockProto blocks = 1;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto> blocks_ =
+        java.util.Collections.emptyList();
+      private void ensureBlocksIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          blocks_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto>(blocks_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+      
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProtoOrBuilder> blocksBuilder_;
+      
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto> getBlocksList() {
+        if (blocksBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(blocks_);
+        } else {
+          return blocksBuilder_.getMessageList();
+        }
+      }
+      public int getBlocksCount() {
+        if (blocksBuilder_ == null) {
+          return blocks_.size();
+        } else {
+          return blocksBuilder_.getCount();
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto getBlocks(int index) {
+        if (blocksBuilder_ == null) {
+          return blocks_.get(index);
+        } else {
+          return blocksBuilder_.getMessage(index);
+        }
+      }
+      public Builder setBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.set(index, value);
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      public Builder setBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addBlocks(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.add(value);
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureBlocksIsMutable();
+          blocks_.add(index, value);
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.add(builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addBlocks(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          blocksBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addAllBlocks(
+          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto> values) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          super.addAll(values, blocks_);
+          onChanged();
+        } else {
+          blocksBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      public Builder clearBlocks() {
+        if (blocksBuilder_ == null) {
+          blocks_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          blocksBuilder_.clear();
+        }
+        return this;
+      }
+      public Builder removeBlocks(int index) {
+        if (blocksBuilder_ == null) {
+          ensureBlocksIsMutable();
+          blocks_.remove(index);
+          onChanged();
+        } else {
+          blocksBuilder_.remove(index);
+        }
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder getBlocksBuilder(
+          int index) {
+        return getBlocksFieldBuilder().getBuilder(index);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProtoOrBuilder getBlocksOrBuilder(
+          int index) {
+        if (blocksBuilder_ == null) {
+          return blocks_.get(index);  } else {
+          return blocksBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProtoOrBuilder> 
+           getBlocksOrBuilderList() {
+        if (blocksBuilder_ != null) {
+          return blocksBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(blocks_);
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder addBlocksBuilder() {
+        return getBlocksFieldBuilder().addBuilder(
+            org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.getDefaultInstance());
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder addBlocksBuilder(
+          int index) {
+        return getBlocksFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.getDefaultInstance());
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder> 
+           getBlocksBuilderList() {
+        return getBlocksFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProtoOrBuilder> 
+          getBlocksFieldBuilder() {
+        if (blocksBuilder_ == null) {
+          blocksBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProtoOrBuilder>(
+                  blocks_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          blocks_ = null;
+        }
+        return blocksBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:ReportBadBlocksRequestProto)
+    }
+    
+    static {
+      defaultInstance = new ReportBadBlocksRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ReportBadBlocksRequestProto)
+  }
+  
+  public interface ReportBadBlocksResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class ReportBadBlocksResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ReportBadBlocksResponseProtoOrBuilder {
+    // Use ReportBadBlocksResponseProto.newBuilder() to construct.
+    private ReportBadBlocksResponseProto(Builder builder) {
+      super(builder);
+    }
+    private ReportBadBlocksResponseProto(boolean noInit) {}
+    
+    private static final ReportBadBlocksResponseProto defaultInstance;
+    public static ReportBadBlocksResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ReportBadBlocksResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReportBadBlocksResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReportBadBlocksResponseProto_fieldAccessorTable;
+    }
+    
+    private void initFields() {
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      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.DatanodeProtocolProtos.ReportBadBlocksResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto) obj;
+      
+      boolean result = true;
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto 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.DatanodeProtocolProtos.ReportBadBlocksResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto 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.DatanodeProtocolProtos.ReportBadBlocksResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto 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.DatanodeProtocolProtos.ReportBadBlocksResponseProto 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.DatanodeProtocolProtos.ReportBadBlocksResponseProto 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.DatanodeProtocolProtos.ReportBadBlocksResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto 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.DatanodeProtocolProtos.ReportBadBlocksResponseProto 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.DatanodeProtocolProtos.ReportBadBlocksResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReportBadBlocksResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_ReportBadBlocksResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.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();
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        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;
+            }
+          }
+        }
+      }
+      
+      
+      // @@protoc_insertion_point(builder_scope:ReportBadBlocksResponseProto)
+    }
+    
+    static {
+      defaultInstance = new ReportBadBlocksResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ReportBadBlocksResponseProto)
+  }
+  
+  public interface CommitBlockSynchronizationRequestProtoOrBuilder
+      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();
+    
+    // required uint64 newGenStamp = 2;
+    boolean hasNewGenStamp();
+    long getNewGenStamp();
+    
+    // required uint64 newLength = 3;
+    boolean hasNewLength();
+    long getNewLength();
+    
+    // required bool closeFile = 4;
+    boolean hasCloseFile();
+    boolean getCloseFile();
+    
+    // required bool deleteBlock = 5;
+    boolean hasDeleteBlock();
+    boolean getDeleteBlock();
+    
+    // repeated .DatanodeIDProto newTaragets = 6;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> 
+        getNewTaragetsList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getNewTaragets(int index);
+    int getNewTaragetsCount();
+    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> 
+        getNewTaragetsOrBuilderList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getNewTaragetsOrBuilder(
+        int index);
+  }
+  public static final class CommitBlockSynchronizationRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements CommitBlockSynchronizationRequestProtoOrBuilder {
+    // Use CommitBlockSynchronizationRequestProto.newBuilder() to construct.
+    private CommitBlockSynchronizationRequestProto(Builder builder) {
+      super(builder);
+    }
+    private CommitBlockSynchronizationRequestProto(boolean noInit) {}
+    
+    private static final CommitBlockSynchronizationRequestProto defaultInstance;
+    public static CommitBlockSynchronizationRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public CommitBlockSynchronizationRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_CommitBlockSynchronizationRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_CommitBlockSynchronizationRequestProto_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_;
+    }
+    
+    // required uint64 newGenStamp = 2;
+    public static final int NEWGENSTAMP_FIELD_NUMBER = 2;
+    private long newGenStamp_;
+    public boolean hasNewGenStamp() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getNewGenStamp() {
+      return newGenStamp_;
+    }
+    
+    // required uint64 newLength = 3;
+    public static final int NEWLENGTH_FIELD_NUMBER = 3;
+    private long newLength_;
+    public boolean hasNewLength() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public long getNewLength() {
+      return newLength_;
+    }
+    
+    // required bool closeFile = 4;
+    public static final int CLOSEFILE_FIELD_NUMBER = 4;
+    private boolean closeFile_;
+    public boolean hasCloseFile() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    public boolean getCloseFile() {
+      return closeFile_;
+    }
+    
+    // required bool deleteBlock = 5;
+    public static final int DELETEBLOCK_FIELD_NUMBER = 5;
+    private boolean deleteBlock_;
+    public boolean hasDeleteBlock() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    public boolean getDeleteBlock() {
+      return deleteBlock_;
+    }
+    
+    // repeated .DatanodeIDProto newTaragets = 6;
+    public static final int NEWTARAGETS_FIELD_NUMBER = 6;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> newTaragets_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> getNewTaragetsList() {
+      return newTaragets_;
+    }
+    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> 
+        getNewTaragetsOrBuilderList() {
+      return newTaragets_;
+    }
+    public int getNewTaragetsCount() {
+      return newTaragets_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getNewTaragets(int index) {
+      return newTaragets_.get(index);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getNewTaragetsOrBuilder(
+        int index) {
+      return newTaragets_.get(index);
+    }
+    
+    private void initFields() {
+      block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+      newGenStamp_ = 0L;
+      newLength_ = 0L;
+      closeFile_ = false;
+      deleteBlock_ = false;
+      newTaragets_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasNewGenStamp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasNewLength()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasCloseFile()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasDeleteBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getBlock().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getNewTaragetsCount(); i++) {
+        if (!getNewTaragets(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, block_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, newGenStamp_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt64(3, newLength_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(4, closeFile_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBool(5, deleteBlock_);
+      }
+      for (int i = 0; i < newTaragets_.size(); i++) {
+        output.writeMessage(6, newTaragets_.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, block_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, newGenStamp_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, newLength_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(4, closeFile_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(5, deleteBlock_);
+      }
+      for (int i = 0; i < newTaragets_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(6, newTaragets_.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.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlock() == other.hasBlock());
+      if (hasBlock()) {
+        result = result && getBlock()
+            .equals(other.getBlock());
+      }
+      result = result && (hasNewGenStamp() == other.hasNewGenStamp());
+      if (hasNewGenStamp()) {
+        result = result && (getNewGenStamp()
+            == other.getNewGenStamp());
+      }
+      result = result && (hasNewLength() == other.hasNewLength());
+      if (hasNewLength()) {
+        result = result && (getNewLength()
+            == other.getNewLength());
+      }
+      result = result && (hasCloseFile() == other.hasCloseFile());
+      if (hasCloseFile()) {
+        result = result && (getCloseFile()
+            == other.getCloseFile());
+      }
+      result = result && (hasDeleteBlock() == other.hasDeleteBlock());
+      if (hasDeleteBlock()) {
+        result = result && (getDeleteBlock()
+            == other.getDeleteBlock());
+      }
+      result = result && getNewTaragetsList()
+          .equals(other.getNewTaragetsList());
+      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 (hasNewGenStamp()) {
+        hash = (37 * hash) + NEWGENSTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getNewGenStamp());
+      }
+      if (hasNewLength()) {
+        hash = (37 * hash) + NEWLENGTH_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getNewLength());
+      }
+      if (hasCloseFile()) {
+        hash = (37 * hash) + CLOSEFILE_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getCloseFile());
+      }
+      if (hasDeleteBlock()) {
+        hash = (37 * hash) + DELETEBLOCK_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getDeleteBlock());
+      }
+      if (getNewTaragetsCount() > 0) {
+        hash = (37 * hash) + NEWTARAGETS_FIELD_NUMBER;
+        hash = (53 * hash) + getNewTaragetsList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_CommitBlockSynchronizationRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_CommitBlockSynchronizationRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBlockFieldBuilder();
+          getNewTaragetsFieldBuilder();
+        }
+      }
+      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);
+        newGenStamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        newLength_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        closeFile_ = false;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        deleteBlock_ = false;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        if (newTaragetsBuilder_ == null) {
+          newTaragets_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000020);
+        } else {
+          newTaragetsBuilder_.clear();
+        }
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto(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;
+        }
+        result.newGenStamp_ = newGenStamp_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.newLength_ = newLength_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.closeFile_ = closeFile_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.deleteBlock_ = deleteBlock_;
+        if (newTaragetsBuilder_ == null) {
+          if (((bitField0_ & 0x00000020) == 0x00000020)) {
+            newTaragets_ = java.util.Collections.unmodifiableList(newTaragets_);
+            bitField0_ = (bitField0_ & ~0x00000020);
+          }
+          result.newTaragets_ = newTaragets_;
+        } else {
+          result.newTaragets_ = newTaragetsBuilder_.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.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto.getDefaultInstance()) return this;
+        if (other.hasBlock()) {
+          mergeBlock(other.getBlock());
+        }
+        if (other.hasNewGenStamp()) {
+          setNewGenStamp(other.getNewGenStamp());
+        }
+        if (other.hasNewLength()) {
+          setNewLength(other.getNewLength());
+        }
+        if (other.hasCloseFile()) {
+          setCloseFile(other.getCloseFile());
+        }
+        if (other.hasDeleteBlock()) {
+          setDeleteBlock(other.getDeleteBlock());
+        }
+        if (newTaragetsBuilder_ == null) {
+          if (!other.newTaragets_.isEmpty()) {
+            if (newTaragets_.isEmpty()) {
+              newTaragets_ = other.newTaragets_;
+              bitField0_ = (bitField0_ & ~0x00000020);
+            } else {
+              ensureNewTaragetsIsMutable();
+              newTaragets_.addAll(other.newTaragets_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.newTaragets_.isEmpty()) {
+            if (newTaragetsBuilder_.isEmpty()) {
+              newTaragetsBuilder_.dispose();
+              newTaragetsBuilder_ = null;
+              newTaragets_ = other.newTaragets_;
+              bitField0_ = (bitField0_ & ~0x00000020);
+              newTaragetsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getNewTaragetsFieldBuilder() : null;
+            } else {
+              newTaragetsBuilder_.addAllMessages(other.newTaragets_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBlock()) {
+          
+          return false;
+        }
+        if (!hasNewGenStamp()) {
+          
+          return false;
+        }
+        if (!hasNewLength()) {
+          
+          return false;
+        }
+        if (!hasCloseFile()) {
+          
+          return false;
+        }
+        if (!hasDeleteBlock()) {
+          
+          return false;
+        }
+        if (!getBlock().isInitialized()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getNewTaragetsCount(); i++) {
+          if (!getNewTaragets(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.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 16: {
+              bitField0_ |= 0x00000002;
+              newGenStamp_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              newLength_ = input.readUInt64();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              closeFile_ = input.readBool();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              deleteBlock_ = input.readBool();
+              break;
+            }
+            case 50: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.newBuilder();
+              input.readMessage(subBuilder, extensionRegistry);
+              addNewTaragets(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_;
+      }
+      
+      // required uint64 newGenStamp = 2;
+      private long newGenStamp_ ;
+      public boolean hasNewGenStamp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getNewGenStamp() {
+        return newGenStamp_;
+      }
+      public Builder setNewGenStamp(long value) {
+        bitField0_ |= 0x00000002;
+        newGenStamp_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearNewGenStamp() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        newGenStamp_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required uint64 newLength = 3;
+      private long newLength_ ;
+      public boolean hasNewLength() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public long getNewLength() {
+        return newLength_;
+      }
+      public Builder setNewLength(long value) {
+        bitField0_ |= 0x00000004;
+        newLength_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearNewLength() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        newLength_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required bool closeFile = 4;
+      private boolean closeFile_ ;
+      public boolean hasCloseFile() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      public boolean getCloseFile() {
+        return closeFile_;
+      }
+      public Builder setCloseFile(boolean value) {
+        bitField0_ |= 0x00000008;
+        closeFile_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearCloseFile() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        closeFile_ = false;
+        onChanged();
+        return this;
+      }
+      
+      // required bool deleteBlock = 5;
+      private boolean deleteBlock_ ;
+      public boolean hasDeleteBlock() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      public boolean getDeleteBlock() {
+        return deleteBlock_;
+      }
+      public Builder setDeleteBlock(boolean value) {
+        bitField0_ |= 0x00000010;
+        deleteBlock_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearDeleteBlock() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        deleteBlock_ = false;
+        onChanged();
+        return this;
+      }
+      
+      // repeated .DatanodeIDProto newTaragets = 6;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> newTaragets_ =
+        java.util.Collections.emptyList();
+      private void ensureNewTaragetsIsMutable() {
+        if (!((bitField0_ & 0x00000020) == 0x00000020)) {
+          newTaragets_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto>(newTaragets_);
+          bitField0_ |= 0x00000020;
+         }
+      }
+      
+      private com.google.protobuf.RepeatedFieldBuilder<
+          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> newTaragetsBuilder_;
+      
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> getNewTaragetsList() {
+        if (newTaragetsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(newTaragets_);
+        } else {
+          return newTaragetsBuilder_.getMessageList();
+        }
+      }
+      public int getNewTaragetsCount() {
+        if (newTaragetsBuilder_ == null) {
+          return newTaragets_.size();
+        } else {
+          return newTaragetsBuilder_.getCount();
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getNewTaragets(int index) {
+        if (newTaragetsBuilder_ == null) {
+          return newTaragets_.get(index);
+        } else {
+          return newTaragetsBuilder_.getMessage(index);
+        }
+      }
+      public Builder setNewTaragets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
+        if (newTaragetsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureNewTaragetsIsMutable();
+          newTaragets_.set(index, value);
+          onChanged();
+        } else {
+          newTaragetsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      public Builder setNewTaragets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder builderForValue) {
+        if (newTaragetsBuilder_ == null) {
+          ensureNewTaragetsIsMutable();
+          newTaragets_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          newTaragetsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addNewTaragets(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
+        if (newTaragetsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureNewTaragetsIsMutable();
+          newTaragets_.add(value);
+          onChanged();
+        } else {
+          newTaragetsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      public Builder addNewTaragets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
+        if (newTaragetsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureNewTaragetsIsMutable();
+          newTaragets_.add(index, value);
+          onChanged();
+        } else {
+          newTaragetsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      public Builder addNewTaragets(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder builderForValue) {
+        if (newTaragetsBuilder_ == null) {
+          ensureNewTaragetsIsMutable();
+          newTaragets_.add(builderForValue.build());
+          onChanged();
+        } else {
+          newTaragetsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addNewTaragets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder builderForValue) {
+        if (newTaragetsBuilder_ == null) {
+          ensureNewTaragetsIsMutable();
+          newTaragets_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          newTaragetsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addAllNewTaragets(
+          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> values) {
+        if (newTaragetsBuilder_ == null) {
+          ensureNewTaragetsIsMutable();
+          super.addAll(values, newTaragets_);
+          onChanged();
+        } else {
+          newTaragetsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      public Builder clearNewTaragets() {
+        if (newTaragetsBuilder_ == null) {
+          newTaragets_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000020);
+          onChanged();
+        } else {
+          newTaragetsBuilder_.clear();
+        }
+        return this;
+      }
+      public Builder removeNewTaragets(int index) {
+        if (newTaragetsBuilder_ == null) {
+          ensureNewTaragetsIsMutable();
+          newTaragets_.remove(index);
+          onChanged();
+        } else {
+          newTaragetsBuilder_.remove(index);
+        }
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder getNewTaragetsBuilder(
+          int index) {
+        return getNewTaragetsFieldBuilder().getBuilder(index);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getNewTaragetsOrBuilder(
+          int index) {
+        if (newTaragetsBuilder_ == null) {
+          return newTaragets_.get(index);  } else {
+          return newTaragetsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> 
+           getNewTaragetsOrBuilderList() {
+        if (newTaragetsBuilder_ != null) {
+          return newTaragetsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(newTaragets_);
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder addNewTaragetsBuilder() {
+        return getNewTaragetsFieldBuilder().addBuilder(
+            org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance());
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder addNewTaragetsBuilder(
+          int index) {
+        return getNewTaragetsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance());
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder> 
+           getNewTaragetsBuilderList() {
+        return getNewTaragetsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          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> 
+          getNewTaragetsFieldBuilder() {
+        if (newTaragetsBuilder_ == null) {
+          newTaragetsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              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>(
+                  newTaragets_,
+                  ((bitField0_ & 0x00000020) == 0x00000020),
+                  getParentForChildren(),
+                  isClean());
+          newTaragets_ = null;
+        }
+        return newTaragetsBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:CommitBlockSynchronizationRequestProto)
+    }
+    
+    static {
+      defaultInstance = new CommitBlockSynchronizationRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:CommitBlockSynchronizationRequestProto)
+  }
+  
+  public interface CommitBlockSynchronizationResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class CommitBlockSynchronizationResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements CommitBlockSynchronizationResponseProtoOrBuilder {
+    // Use CommitBlockSynchronizationResponseProto.newBuilder() to construct.
+    private CommitBlockSynchronizationResponseProto(Builder builder) {
+      super(builder);
+    }
+    private CommitBlockSynchronizationResponseProto(boolean noInit) {}
+    
+    private static final CommitBlockSynchronizationResponseProto defaultInstance;
+    public static CommitBlockSynchronizationResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public CommitBlockSynchronizationResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_CommitBlockSynchronizationResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_CommitBlockSynchronizationResponseProto_fieldAccessorTable;
+    }
+    
+    private void initFields() {
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      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.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto) obj;
+      
+      boolean result = true;
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto 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.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_CommitBlockSynchronizationResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.internal_static_CommitBlockSynchronizationResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.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();
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        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;
+            }
+          }
+        }
+      }
+      
+      
+      // @@protoc_insertion_point(builder_scope:CommitBlockSynchronizationResponseProto)
+    }
+    
+    static {
+      defaultInstance = new CommitBlockSynchronizationResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:CommitBlockSynchronizationResponseProto)
+  }
+  
+  public static abstract class DatanodeProtocolService
+      implements com.google.protobuf.Service {
+    protected DatanodeProtocolService() {}
+    
+    public interface Interface {
+      public abstract void registerDatanode(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto> done);
+      
+      public abstract void sendHeartbeat(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto> done);
+      
+      public abstract void blockReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto> done);
+      
+      public abstract void blockReceivedAndDeleted(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto> done);
+      
+      public abstract void errorReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto> done);
+      
+      public abstract void processUpgrade(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto> done);
+      
+      public abstract void reportBadBlocks(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto> done);
+      
+      public abstract void commitBlockSynchronization(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto> done);
+      
+    }
+    
+    public static com.google.protobuf.Service newReflectiveService(
+        final Interface impl) {
+      return new DatanodeProtocolService() {
+        @java.lang.Override
+        public  void registerDatanode(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto> done) {
+          impl.registerDatanode(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void sendHeartbeat(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto> done) {
+          impl.sendHeartbeat(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void blockReport(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto> done) {
+          impl.blockReport(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void blockReceivedAndDeleted(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto> done) {
+          impl.blockReceivedAndDeleted(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void errorReport(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto> done) {
+          impl.errorReport(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void processUpgrade(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto> done) {
+          impl.processUpgrade(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void reportBadBlocks(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto> done) {
+          impl.reportBadBlocks(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void commitBlockSynchronization(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto> done) {
+          impl.commitBlockSynchronization(controller, request, done);
+        }
+        
+      };
+    }
+    
+    public static com.google.protobuf.BlockingService
+        newReflectiveBlockingService(final BlockingInterface impl) {
+      return new com.google.protobuf.BlockingService() {
+        public final com.google.protobuf.Descriptors.ServiceDescriptor
+            getDescriptorForType() {
+          return getDescriptor();
+        }
+        
+        public final com.google.protobuf.Message callBlockingMethod(
+            com.google.protobuf.Descriptors.MethodDescriptor method,
+            com.google.protobuf.RpcController controller,
+            com.google.protobuf.Message request)
+            throws com.google.protobuf.ServiceException {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.callBlockingMethod() given method descriptor for " +
+              "wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return impl.registerDatanode(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto)request);
+            case 1:
+              return impl.sendHeartbeat(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto)request);
+            case 2:
+              return impl.blockReport(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto)request);
+            case 3:
+              return impl.blockReceivedAndDeleted(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto)request);
+            case 4:
+              return impl.errorReport(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto)request);
+            case 5:
+              return impl.processUpgrade(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto)request);
+            case 6:
+              return impl.reportBadBlocks(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto)request);
+            case 7:
+              return impl.commitBlockSynchronization(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto)request);
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+        public final com.google.protobuf.Message
+            getRequestPrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getRequestPrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto.getDefaultInstance();
+            case 2:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto.getDefaultInstance();
+            case 3:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto.getDefaultInstance();
+            case 4:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto.getDefaultInstance();
+            case 5:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto.getDefaultInstance();
+            case 6:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto.getDefaultInstance();
+            case 7:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+        public final com.google.protobuf.Message
+            getResponsePrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getResponsePrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.getDefaultInstance();
+            case 2:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.getDefaultInstance();
+            case 3:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.getDefaultInstance();
+            case 4:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance();
+            case 5:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.getDefaultInstance();
+            case 6:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.getDefaultInstance();
+            case 7:
+              return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+      };
+    }
+    
+    public abstract void registerDatanode(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto> done);
+    
+    public abstract void sendHeartbeat(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto> done);
+    
+    public abstract void blockReport(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto> done);
+    
+    public abstract void blockReceivedAndDeleted(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto> done);
+    
+    public abstract void errorReport(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto> done);
+    
+    public abstract void processUpgrade(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto> done);
+    
+    public abstract void reportBadBlocks(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto> done);
+    
+    public abstract void commitBlockSynchronization(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto> done);
+    
+    public static final
+        com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.getDescriptor().getServices().get(0);
+    }
+    public final com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    
+    public final void callMethod(
+        com.google.protobuf.Descriptors.MethodDescriptor method,
+        com.google.protobuf.RpcController controller,
+        com.google.protobuf.Message request,
+        com.google.protobuf.RpcCallback<
+          com.google.protobuf.Message> done) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.callMethod() given method descriptor for wrong " +
+          "service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          this.registerDatanode(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto>specializeCallback(
+              done));
+          return;
+        case 1:
+          this.sendHeartbeat(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto>specializeCallback(
+              done));
+          return;
+        case 2:
+          this.blockReport(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto>specializeCallback(
+              done));
+          return;
+        case 3:
+          this.blockReceivedAndDeleted(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto>specializeCallback(
+              done));
+          return;
+        case 4:
+          this.errorReport(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto>specializeCallback(
+              done));
+          return;
+        case 5:
+          this.processUpgrade(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto>specializeCallback(
+              done));
+          return;
+        case 6:
+          this.reportBadBlocks(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto>specializeCallback(
+              done));
+          return;
+        case 7:
+          this.commitBlockSynchronization(controller, (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto>specializeCallback(
+              done));
+          return;
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+    
+    public final com.google.protobuf.Message
+        getRequestPrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getRequestPrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto.getDefaultInstance();
+        case 2:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto.getDefaultInstance();
+        case 3:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto.getDefaultInstance();
+        case 4:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto.getDefaultInstance();
+        case 5:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto.getDefaultInstance();
+        case 6:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto.getDefaultInstance();
+        case 7:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+    
+    public final com.google.protobuf.Message
+        getResponsePrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getResponsePrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.getDefaultInstance();
+        case 2:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.getDefaultInstance();
+        case 3:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.getDefaultInstance();
+        case 4:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance();
+        case 5:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.getDefaultInstance();
+        case 6:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.getDefaultInstance();
+        case 7:
+          return org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+    
+    public static Stub newStub(
+        com.google.protobuf.RpcChannel channel) {
+      return new Stub(channel);
+    }
+    
+    public static final class Stub extends org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeProtocolService implements Interface {
+      private Stub(com.google.protobuf.RpcChannel channel) {
+        this.channel = channel;
+      }
+      
+      private final com.google.protobuf.RpcChannel channel;
+      
+      public com.google.protobuf.RpcChannel getChannel() {
+        return channel;
+      }
+      
+      public  void registerDatanode(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.getDefaultInstance()));
+      }
+      
+      public  void sendHeartbeat(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.getDefaultInstance()));
+      }
+      
+      public  void blockReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(2),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.getDefaultInstance()));
+      }
+      
+      public  void blockReceivedAndDeleted(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(3),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.getDefaultInstance()));
+      }
+      
+      public  void errorReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(4),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance()));
+      }
+      
+      public  void processUpgrade(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(5),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.getDefaultInstance()));
+      }
+      
+      public  void reportBadBlocks(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(6),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.getDefaultInstance()));
+      }
+      
+      public  void commitBlockSynchronization(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(7),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.getDefaultInstance()));
+      }
+    }
+    
+    public static BlockingInterface newBlockingStub(
+        com.google.protobuf.BlockingRpcChannel channel) {
+      return new BlockingStub(channel);
+    }
+    
+    public interface BlockingInterface {
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto registerDatanode(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto sendHeartbeat(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto blockReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto errorReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto processUpgrade(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto reportBadBlocks(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto commitBlockSynchronization(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto request)
+          throws com.google.protobuf.ServiceException;
+    }
+    
+    private static final class BlockingStub implements BlockingInterface {
+      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+        this.channel = channel;
+      }
+      
+      private final com.google.protobuf.BlockingRpcChannel channel;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto registerDatanode(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto sendHeartbeat(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto blockReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(2),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(3),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto errorReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(4),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto processUpgrade(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(5),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto reportBadBlocks(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(6),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto commitBlockSynchronization(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(7),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.getDefaultInstance());
+      }
+      
+    }
+  }
+  
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DatanodeRegistrationProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DatanodeRegistrationProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DatanodeCommandProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DatanodeCommandProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BalancerBandwidthCommandProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BalancerBandwidthCommandProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BlockCommandProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BlockCommandProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BlockRecoveryCommndProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BlockRecoveryCommndProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_FinalizeCommandProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_FinalizeCommandProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_KeyUpdateCommandProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_KeyUpdateCommandProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_RegisterCommandProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_RegisterCommandProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_UpgradeCommandProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_UpgradeCommandProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_RegisterDatanodeRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_RegisterDatanodeRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_RegisterDatanodeResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_RegisterDatanodeResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_HeartbeatRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_HeartbeatRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_HeartbeatResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_HeartbeatResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BlockReportRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BlockReportRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BlockReportResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BlockReportResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ReceivedDeletedBlockInfoProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ReceivedDeletedBlockInfoProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BlockReceivedAndDeletedRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BlockReceivedAndDeletedRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BlockReceivedAndDeletedResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BlockReceivedAndDeletedResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ErrorReportRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ErrorReportRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ErrorReportResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ErrorReportResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ProcessUpgradeRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ProcessUpgradeRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ProcessUpgradeResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ProcessUpgradeResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ReportBadBlocksRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ReportBadBlocksRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ReportBadBlocksResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ReportBadBlocksResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_CommitBlockSynchronizationRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_CommitBlockSynchronizationRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_CommitBlockSynchronizationResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_CommitBlockSynchronizationResponseProto_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\026DatanodeProtocol.proto\032\nhdfs.proto\"\220\001\n" +
+      "\031DatanodeRegistrationProto\022$\n\ndatanodeID" +
+      "\030\001 \002(\0132\020.DatanodeIDProto\022&\n\013storateInfo\030" +
+      "\002 \002(\0132\021.StorageInfoProto\022%\n\004keys\030\003 \002(\0132\027" +
+      ".ExportedBlockKeysProto\"\243\004\n\024DatanodeComm" +
+      "andProto\022+\n\007cmdType\030\001 \002(\0162\032.DatanodeComm" +
+      "andProto.Type\0223\n\013balancerCmd\030\002 \001(\0132\036.Bal" +
+      "ancerBandwidthCommandProto\022\"\n\006blkCmd\030\003 \001" +
+      "(\0132\022.BlockCommandProto\022.\n\013recoveryCmd\030\004 " +
+      "\001(\0132\031.BlockRecoveryCommndProto\022*\n\013finali",
+      "zeCmd\030\005 \001(\0132\025.FinalizeCommandProto\022,\n\014ke" +
+      "yUpdateCmd\030\006 \001(\0132\026.KeyUpdateCommandProto" +
+      "\022*\n\013registerCmd\030\007 \001(\0132\025.RegisterCommandP" +
+      "roto\022(\n\nupgradeCmd\030\010 \001(\0132\024.UpgradeComman" +
+      "dProto\"\244\001\n\004Type\022\034\n\030BalancerBandwidthComm" +
+      "and\020\000\022\020\n\014BlockCommand\020\001\022\030\n\024BlockRecovery" +
+      "Command\020\002\022\023\n\017FinalizeCommand\020\003\022\024\n\020KeyUpd" +
+      "ateCommand\020\004\022\023\n\017RegisterCommand\020\005\022\022\n\016Upg" +
+      "radeCommand\020\006\"2\n\035BalancerBandwidthComman" +
+      "dProto\022\021\n\tbandwidth\030\001 \002(\004\"\274\001\n\021BlockComma",
+      "ndProto\022\016\n\006action\030\001 \002(\r\022\023\n\013blockPoolId\030\002" +
+      " \002(\t\022\033\n\006blocks\030\003 \003(\0132\013.BlockProto\022\"\n\007tar" +
+      "gets\030\004 \003(\0132\021.DatanodeIDsProto\"A\n\006Action\022" +
+      "\013\n\007UNKNOWN\020\000\022\014\n\010TRANSFER\020\001\022\016\n\nINVALIDATE" +
+      "\020\002\022\014\n\010SHUTDOWN\020\003\"A\n\030BlockRecoveryCommndP" +
+      "roto\022%\n\006blocks\030\001 \003(\0132\025.RecoveringBlockPr" +
+      "oto\"+\n\024FinalizeCommandProto\022\023\n\013blockPool" +
+      "Id\030\001 \002(\t\">\n\025KeyUpdateCommandProto\022%\n\004key" +
+      "s\030\001 \002(\0132\027.ExportedBlockKeysProto\"\026\n\024Regi" +
+      "sterCommandProto\"\212\001\n\023UpgradeCommandProto",
+      "\022\016\n\006action\030\001 \002(\r\022\017\n\007version\030\002 \002(\r\022\025\n\rupg" +
+      "radeStatus\030\003 \002(\r\";\n\006Action\022\013\n\007UNKNOWN\020\000\022" +
+      "\021\n\rREPORT_STATUS\020d\022\021\n\rSTART_UPGRADE\020e\"P\n" +
+      "\034RegisterDatanodeRequestProto\0220\n\014registr" +
+      "ation\030\001 \002(\0132\032.DatanodeRegistrationProto\"" +
+      "Q\n\035RegisterDatanodeResponseProto\0220\n\014regi" +
+      "stration\030\001 \002(\0132\032.DatanodeRegistrationPro" +
+      "to\"\334\001\n\025HeartbeatRequestProto\0220\n\014registra" +
+      "tion\030\001 \002(\0132\032.DatanodeRegistrationProto\022\020" +
+      "\n\010capacity\030\002 \002(\004\022\017\n\007dfsUsed\030\003 \002(\004\022\021\n\trem",
+      "aining\030\004 \002(\004\022\025\n\rblockPoolUsed\030\005 \002(\004\022\027\n\017x" +
+      "mitsInProgress\030\006 \002(\r\022\024\n\014xceiverCount\030\007 \002" +
+      "(\r\022\025\n\rfailedVolumes\030\010 \002(\r\"=\n\026HeartbeatRe" +
+      "sponseProto\022#\n\004cmds\030\001 \003(\0132\025.DatanodeComm" +
+      "andProto\"t\n\027BlockReportRequestProto\0220\n\014r" +
+      "egistration\030\001 \002(\0132\032.DatanodeRegistration" +
+      "Proto\022\023\n\013blockPoolId\030\002 \002(\t\022\022\n\006blocks\030\003 \003" +
+      "(\004B\002\020\001\">\n\030BlockReportResponseProto\022\"\n\003cm" +
+      "d\030\001 \002(\0132\025.DatanodeCommandProto\"O\n\035Receiv" +
+      "edDeletedBlockInfoProto\022\032\n\005block\030\001 \002(\0132\013",
+      ".BlockProto\022\022\n\ndeleteHint\030\002 \001(\t\"\234\001\n#Bloc" +
+      "kReceivedAndDeletedRequestProto\0220\n\014regis" +
+      "tration\030\001 \002(\0132\032.DatanodeRegistrationProt" +
+      "o\022\023\n\013blockPoolId\030\002 \002(\t\022.\n\006blocks\030\003 \003(\0132\036" +
+      ".ReceivedDeletedBlockInfoProto\"&\n$BlockR" +
+      "eceivedAndDeletedResponseProto\"\275\001\n\027Error" +
+      "ReportRequestProto\0220\n\014registartion\030\001 \002(\013" +
+      "2\032.DatanodeRegistrationProto\022\021\n\terrorCod" +
+      "e\030\002 \002(\r\022\013\n\003msg\030\003 \002(\t\"P\n\tErrorCode\022\n\n\006NOT" +
+      "IFY\020\000\022\016\n\nDISK_ERROR\020\001\022\021\n\rINVALID_BLOCK\020\002",
+      "\022\024\n\020FATAL_DISK_ERROR\020\003\"\032\n\030ErrorReportRes" +
+      "ponseProto\"?\n\032ProcessUpgradeRequestProto" +
+      "\022!\n\003cmd\030\001 \001(\0132\024.UpgradeCommandProto\"@\n\033P" +
+      "rocessUpgradeResponseProto\022!\n\003cmd\030\001 \001(\0132" +
+      "\024.UpgradeCommandProto\"A\n\033ReportBadBlocks" +
+      "RequestProto\022\"\n\006blocks\030\001 \003(\0132\022.LocatedBl" +
+      "ockProto\"\036\n\034ReportBadBlocksResponseProto" +
+      "\"\303\001\n&CommitBlockSynchronizationRequestPr" +
+      "oto\022\"\n\005block\030\001 \002(\0132\023.ExtendedBlockProto\022" +
+      "\023\n\013newGenStamp\030\002 \002(\004\022\021\n\tnewLength\030\003 \002(\004\022",
+      "\021\n\tcloseFile\030\004 \002(\010\022\023\n\013deleteBlock\030\005 \002(\010\022" +
+      "%\n\013newTaragets\030\006 \003(\0132\020.DatanodeIDProto\")" +
+      "\n\'CommitBlockSynchronizationResponseProt" +
+      "o2\254\005\n\027DatanodeProtocolService\022Q\n\020registe" +
+      "rDatanode\022\035.RegisterDatanodeRequestProto" +
+      "\032\036.RegisterDatanodeResponseProto\022@\n\rsend" +
+      "Heartbeat\022\026.HeartbeatRequestProto\032\027.Hear" +
+      "tbeatResponseProto\022B\n\013blockReport\022\030.Bloc" +
+      "kReportRequestProto\032\031.BlockReportRespons" +
+      "eProto\022f\n\027blockReceivedAndDeleted\022$.Bloc",
+      "kReceivedAndDeletedRequestProto\032%.BlockR" +
+      "eceivedAndDeletedResponseProto\022B\n\013errorR" +
+      "eport\022\030.ErrorReportRequestProto\032\031.ErrorR" +
+      "eportResponseProto\022K\n\016processUpgrade\022\033.P" +
+      "rocessUpgradeRequestProto\032\034.ProcessUpgra" +
+      "deResponseProto\022N\n\017reportBadBlocks\022\034.Rep" +
+      "ortBadBlocksRequestProto\032\035.ReportBadBloc" +
+      "ksResponseProto\022o\n\032commitBlockSynchroniz" +
+      "ation\022\'.CommitBlockSynchronizationReques" +
+      "tProto\032(.CommitBlockSynchronizationRespo",
+      "nseProtoBE\n%org.apache.hadoop.hdfs.proto" +
+      "col.protoB\026DatanodeProtocolProtos\210\001\001\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_DatanodeRegistrationProto_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_DatanodeRegistrationProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_DatanodeRegistrationProto_descriptor,
+              new java.lang.String[] { "DatanodeID", "StorateInfo", "Keys", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto.Builder.class);
+          internal_static_DatanodeCommandProto_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_DatanodeCommandProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_DatanodeCommandProto_descriptor,
+              new java.lang.String[] { "CmdType", "BalancerCmd", "BlkCmd", "RecoveryCmd", "FinalizeCmd", "KeyUpdateCmd", "RegisterCmd", "UpgradeCmd", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto.Builder.class);
+          internal_static_BalancerBandwidthCommandProto_descriptor =
+            getDescriptor().getMessageTypes().get(2);
+          internal_static_BalancerBandwidthCommandProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BalancerBandwidthCommandProto_descriptor,
+              new java.lang.String[] { "Bandwidth", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto.Builder.class);
+          internal_static_BlockCommandProto_descriptor =
+            getDescriptor().getMessageTypes().get(3);
+          internal_static_BlockCommandProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BlockCommandProto_descriptor,
+              new java.lang.String[] { "Action", "BlockPoolId", "Blocks", "Targets", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto.Builder.class);
+          internal_static_BlockRecoveryCommndProto_descriptor =
+            getDescriptor().getMessageTypes().get(4);
+          internal_static_BlockRecoveryCommndProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BlockRecoveryCommndProto_descriptor,
+              new java.lang.String[] { "Blocks", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommndProto.Builder.class);
+          internal_static_FinalizeCommandProto_descriptor =
+            getDescriptor().getMessageTypes().get(5);
+          internal_static_FinalizeCommandProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_FinalizeCommandProto_descriptor,
+              new java.lang.String[] { "BlockPoolId", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto.Builder.class);
+          internal_static_KeyUpdateCommandProto_descriptor =
+            getDescriptor().getMessageTypes().get(6);
+          internal_static_KeyUpdateCommandProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_KeyUpdateCommandProto_descriptor,
+              new java.lang.String[] { "Keys", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto.Builder.class);
+          internal_static_RegisterCommandProto_descriptor =
+            getDescriptor().getMessageTypes().get(7);
+          internal_static_RegisterCommandProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_RegisterCommandProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto.Builder.class);
+          internal_static_UpgradeCommandProto_descriptor =
+            getDescriptor().getMessageTypes().get(8);
+          internal_static_UpgradeCommandProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_UpgradeCommandProto_descriptor,
+              new java.lang.String[] { "Action", "Version", "UpgradeStatus", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto.Builder.class);
+          internal_static_RegisterDatanodeRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(9);
+          internal_static_RegisterDatanodeRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_RegisterDatanodeRequestProto_descriptor,
+              new java.lang.String[] { "Registration", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto.Builder.class);
+          internal_static_RegisterDatanodeResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(10);
+          internal_static_RegisterDatanodeResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_RegisterDatanodeResponseProto_descriptor,
+              new java.lang.String[] { "Registration", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto.Builder.class);
+          internal_static_HeartbeatRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(11);
+          internal_static_HeartbeatRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_HeartbeatRequestProto_descriptor,
+              new java.lang.String[] { "Registration", "Capacity", "DfsUsed", "Remaining", "BlockPoolUsed", "XmitsInProgress", "XceiverCount", "FailedVolumes", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto.Builder.class);
+          internal_static_HeartbeatResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(12);
+          internal_static_HeartbeatResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_HeartbeatResponseProto_descriptor,
+              new java.lang.String[] { "Cmds", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto.Builder.class);
+          internal_static_BlockReportRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(13);
+          internal_static_BlockReportRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BlockReportRequestProto_descriptor,
+              new java.lang.String[] { "Registration", "BlockPoolId", "Blocks", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto.Builder.class);
+          internal_static_BlockReportResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(14);
+          internal_static_BlockReportResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BlockReportResponseProto_descriptor,
+              new java.lang.String[] { "Cmd", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto.Builder.class);
+          internal_static_ReceivedDeletedBlockInfoProto_descriptor =
+            getDescriptor().getMessageTypes().get(15);
+          internal_static_ReceivedDeletedBlockInfoProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ReceivedDeletedBlockInfoProto_descriptor,
+              new java.lang.String[] { "Block", "DeleteHint", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto.Builder.class);
+          internal_static_BlockReceivedAndDeletedRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(16);
+          internal_static_BlockReceivedAndDeletedRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BlockReceivedAndDeletedRequestProto_descriptor,
+              new java.lang.String[] { "Registration", "BlockPoolId", "Blocks", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto.Builder.class);
+          internal_static_BlockReceivedAndDeletedResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(17);
+          internal_static_BlockReceivedAndDeletedResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BlockReceivedAndDeletedResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto.Builder.class);
+          internal_static_ErrorReportRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(18);
+          internal_static_ErrorReportRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ErrorReportRequestProto_descriptor,
+              new java.lang.String[] { "Registartion", "ErrorCode", "Msg", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto.Builder.class);
+          internal_static_ErrorReportResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(19);
+          internal_static_ErrorReportResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ErrorReportResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto.Builder.class);
+          internal_static_ProcessUpgradeRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(20);
+          internal_static_ProcessUpgradeRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ProcessUpgradeRequestProto_descriptor,
+              new java.lang.String[] { "Cmd", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto.Builder.class);
+          internal_static_ProcessUpgradeResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(21);
+          internal_static_ProcessUpgradeResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ProcessUpgradeResponseProto_descriptor,
+              new java.lang.String[] { "Cmd", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto.Builder.class);
+          internal_static_ReportBadBlocksRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(22);
+          internal_static_ReportBadBlocksRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ReportBadBlocksRequestProto_descriptor,
+              new java.lang.String[] { "Blocks", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto.Builder.class);
+          internal_static_ReportBadBlocksResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(23);
+          internal_static_ReportBadBlocksResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ReportBadBlocksResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto.Builder.class);
+          internal_static_CommitBlockSynchronizationRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(24);
+          internal_static_CommitBlockSynchronizationRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_CommitBlockSynchronizationRequestProto_descriptor,
+              new java.lang.String[] { "Block", "NewGenStamp", "NewLength", "CloseFile", "DeleteBlock", "NewTaragets", },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto.Builder.class);
+          internal_static_CommitBlockSynchronizationResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(25);
+          internal_static_CommitBlockSynchronizationResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_CommitBlockSynchronizationResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto.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)
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/browseBlock.jsp

@@ -27,7 +27,7 @@
   //for java.io.Serializable
   private static final long serialVersionUID = 1L;
 %>
-
+<!DOCTYPE html>
 <html>
 <head>
 <%JspHelper.createTitle(out, request, request.getParameter("filename")); %>

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/browseDirectory.jsp

@@ -29,7 +29,7 @@
   //for java.io.Serializable
   private static final long serialVersionUID = 1L;
 %>
-
+<!DOCTYPE html>
 <html>
 <head>
 <style type=text/css>

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/tail.jsp

@@ -27,6 +27,7 @@
   //for java.io.Serializable
   private static final long serialVersionUID = 1L;
 %>
+<!DOCTYPE html>
 <html>
 <head>
 <%JspHelper.createTitle(out, request, request.getParameter("filename")); %>

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp

@@ -38,6 +38,7 @@
   int corruptFileCount = corruptFileBlocks.size();
 %>
 
+<!DOCTYPE html>
 <html>
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">
 <title>Hadoop <%=namenodeRole%> <%=namenodeLabel%></title>

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp

@@ -33,6 +33,7 @@
   String namenodeLabel = nn.getNameNodeAddress().getHostName() + ":" + nn.getNameNodeAddress().getPort();
 %>
 
+<!DOCTYPE html>
 <html>
 
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp

@@ -33,6 +33,7 @@ FSNamesystem fsn = nn.getNamesystem();
 String namenodeLabel = nn.getNameNodeAddress().getHostName() + ":" + nn.getNameNodeAddress().getPort();
 %>
 
+<!DOCTYPE html>
 <html>
 
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/nn_browsedfscontent.jsp

@@ -27,6 +27,7 @@
   //for java.io.Serializable
   private static final long serialVersionUID = 1L;
 %>
+<!DOCTYPE html>
 <html>
 
 <title></title>

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.jsp

@@ -27,6 +27,7 @@
   private static final long serialVersionUID = 1L;
 %>
 
+<!DOCTYPE html>
 <html>
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">
 <title>Hadoop SecondaryNameNode</title>

+ 344 - 0
hadoop-hdfs-project/hadoop-hdfs/src/proto/DatanodeProtocol.proto

@@ -0,0 +1,344 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "DatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Information to identify a datanode to a namenode
+ */
+message DatanodeRegistrationProto {
+  required DatanodeIDProto datanodeID = 1;    // Datanode information
+  required StorageInfoProto storateInfo = 2;  // Node information
+  required ExportedBlockKeysProto keys = 3;   // Block keys
+}
+
+/**
+ * Commands sent from namenode to the datanodes
+ */
+message DatanodeCommandProto {
+  enum Type {
+    BalancerBandwidthCommand = 0;
+    BlockCommand = 1;
+    BlockRecoveryCommand = 2;
+    FinalizeCommand = 3;
+    KeyUpdateCommand = 4;
+    RegisterCommand = 5;
+    UpgradeCommand = 6;
+  }
+
+  required Type cmdType = 1;    // Type of the command
+
+  // One of the following command is available when the corresponding
+  // cmdType is set
+  optional BalancerBandwidthCommandProto balancerCmd = 2;
+  optional BlockCommandProto blkCmd = 3;
+  optional BlockRecoveryCommndProto recoveryCmd = 4;
+  optional FinalizeCommandProto finalizeCmd = 5;
+  optional KeyUpdateCommandProto keyUpdateCmd = 6;
+  optional RegisterCommandProto registerCmd = 7;
+  optional UpgradeCommandProto upgradeCmd = 8;
+}
+
+/**
+ * Command sent from namenode to datanode to set the
+ * maximum bandwidth to be used for balancing.
+ */
+message BalancerBandwidthCommandProto {
+
+  // Maximum bandwidth to be used by datanode for balancing
+  required uint64 bandwidth = 1;
+}
+
+/**
+ * Command to instruct datanodes to perform certain action
+ * on the given set of blocks.
+ */
+message BlockCommandProto {
+  enum Action {
+    UNKNOWN = 0;    // Unknown action   
+    TRANSFER = 1;   // Transfer blocks to another datanode
+    INVALIDATE = 2; // Invalidate blocks
+    SHUTDOWN = 3;   // Shutdown node
+  }
+  required uint32 action = 1;
+  required string blockPoolId = 2;
+  repeated BlockProto blocks = 3;
+  repeated DatanodeIDsProto targets = 4;
+}
+
+/**
+ * List of blocks to be recovered by the datanode
+ */
+message BlockRecoveryCommndProto {
+  repeated RecoveringBlockProto blocks = 1;
+}
+
+/**
+ * Finalize the upgrade at the datanode
+ */
+message FinalizeCommandProto {
+  required string blockPoolId = 1; // Block pool to be finalized
+}
+
+/**
+ * Update the block keys at the datanode
+ */
+message KeyUpdateCommandProto {
+  required ExportedBlockKeysProto keys = 1;
+}
+
+/**
+ * Instruct datanode to register with the namenode
+ */
+message RegisterCommandProto {
+  // void
+}
+
+/**
+ * Generic distributed upgrade Command
+ */
+message UpgradeCommandProto {
+  enum Action {
+    UNKNOWN = 0;          // Unknown action
+    REPORT_STATUS = 100;  // Report upgrade status
+    START_UPGRADE = 101;  // Start upgrade
+  }
+  required uint32 action = 1;  // Upgrade action
+  required uint32 version = 2; // Version of the upgrade
+  required uint32 upgradeStatus = 3; // % completed in range 0 & 100
+}
+
+/**
+ * registration - Information of the datanode registering with the namenode
+ */
+message RegisterDatanodeRequestProto {
+  required DatanodeRegistrationProto registration = 1; // Datanode info
+}
+
+/**
+ * registration - Update registration of the datanode that successfully 
+ *                registered. StorageInfo will be updated to include new 
+ *                storage ID if the datanode did not have one in the request.
+ */
+message RegisterDatanodeResponseProto {
+  required DatanodeRegistrationProto registration = 1; // Datanode info
+}
+
+/**
+ * registration - datanode registration information
+ * capacity - total storage capacity available at the datanode
+ * dfsUsed - storage used by HDFS
+ * remaining - remaining storage available for HDFS
+ * blockPoolUsed - storage used by the block pool
+ * xmitsInProgress - number of transfers from this datanode to others
+ * xceiverCount - number of active transceiver threads
+ * failedVolumes - number of failed volumes
+ */
+message HeartbeatRequestProto {
+  required DatanodeRegistrationProto registration = 1; // Datanode info
+  required uint64 capacity = 2;
+  required uint64 dfsUsed = 3;
+  required uint64 remaining = 4;
+  required uint64 blockPoolUsed = 5;
+  required uint32 xmitsInProgress = 6;
+  required uint32 xceiverCount = 7;
+  required uint32 failedVolumes = 8;
+}
+
+/**
+ * cmds - Commands from namenode to datanode.
+ */
+message HeartbeatResponseProto {
+  repeated DatanodeCommandProto cmds = 1;
+}
+
+/**
+ * registration - datanode registration information
+ * blockPoolID  - block pool ID of the reported blocks
+ * blocks       - each block is represented as two longs in the array.
+ *                first long represents block ID
+ *                second long represents length
+ */
+message BlockReportRequestProto {
+  required DatanodeRegistrationProto registration = 1;
+  required string blockPoolId = 2;
+  repeated uint64 blocks = 3 [packed=true];
+}
+
+/**
+ * cmd - Command from namenode to the datanode
+ */
+message BlockReportResponseProto {
+  required DatanodeCommandProto cmd = 1;
+} 
+
+/**
+ * Data structure to send received or deleted block information
+ * from datanode to namenode.
+ *
+ * deleteHint set to "-" indicates block deletion.
+ * other deleteHint indicates block addition.
+ */
+message ReceivedDeletedBlockInfoProto {
+  required BlockProto block = 1;
+  optional string deleteHint = 2;
+}
+
+/**
+ * registration - datanode registration information
+ * blockPoolID  - block pool ID of the reported blocks
+ * blocks       - Received/deleted block list
+ */
+message BlockReceivedAndDeletedRequestProto {
+  required DatanodeRegistrationProto registration = 1;
+  required string blockPoolId = 2;
+  repeated ReceivedDeletedBlockInfoProto blocks = 3;
+}
+
+/**
+ * void response
+ */
+message BlockReceivedAndDeletedResponseProto {
+}
+
+/**
+ * registartion - Datanode reporting the error
+ * errorCode - error code indicating the error
+ * msg - Free text description of the error
+ */
+message ErrorReportRequestProto {
+  enum ErrorCode {
+    NOTIFY = 0;           // Error report to be logged at the namenode
+    DISK_ERROR = 1;       // DN has disk errors but still has valid volumes
+    INVALID_BLOCK = 2;    // Command from namenode has invalid block ID
+    FATAL_DISK_ERROR = 3; // No valid volumes left on datanode
+  }
+  required DatanodeRegistrationProto registartion = 1; // Registartion info
+  required uint32 errorCode = 2;  // Error code
+  required string msg = 3;        // Error message
+}
+
+/**
+ * void response
+ */
+message ErrorReportResponseProto {
+}
+
+/**
+ * cmd - Upgrade command sent from datanode to namenode
+ */
+message ProcessUpgradeRequestProto {
+  optional UpgradeCommandProto cmd = 1;
+}
+
+/**
+ * cmd - Upgrade command sent from namenode to datanode
+ */
+message ProcessUpgradeResponseProto {
+  optional UpgradeCommandProto cmd = 1;
+}
+
+/**
+ * blocks - list of blocks that are reported as corrupt
+ */
+message ReportBadBlocksRequestProto {
+  repeated LocatedBlockProto blocks = 1;
+}
+
+/**
+ * void response
+ */
+message ReportBadBlocksResponseProto {
+}
+
+/**
+ * Commit block synchronization request during lease recovery
+ */
+message CommitBlockSynchronizationRequestProto {
+  required ExtendedBlockProto block = 1;
+  required uint64 newGenStamp = 2;
+  required uint64 newLength = 3;
+  required bool closeFile = 4;
+  required bool deleteBlock = 5;
+  repeated DatanodeIDProto newTaragets = 6;
+}
+
+/**
+ * void response
+ */
+message CommitBlockSynchronizationResponseProto {
+}
+
+/**
+ * Protocol used from datanode to the namenode
+ * See the request and response for details of rpc call.
+ */
+service DatanodeProtocolService {
+  /**
+   * Register a datanode at a namenode
+   */
+  rpc registerDatanode(RegisterDatanodeRequestProto)
+      returns(RegisterDatanodeResponseProto);
+
+  /**
+   * Send heartbeat from datanode to namenode
+   */
+  rpc sendHeartbeat(HeartbeatRequestProto) returns(HeartbeatResponseProto);
+
+  /**
+   * Report blocks at a given datanode to the namenode
+   */
+  rpc blockReport(BlockReportRequestProto) returns(BlockReportResponseProto);
+
+  /**
+   * Report from datanode about recently received or deleted block
+   */
+  rpc blockReceivedAndDeleted(BlockReceivedAndDeletedRequestProto) 
+      returns(BlockReceivedAndDeletedResponseProto);
+
+  /**
+   * Report from a datanode of an error to the active namenode.
+   * Used for debugging.
+   */
+  rpc errorReport(ErrorReportRequestProto) returns(ErrorReportResponseProto);
+
+  /**
+   * Generic way to send commands from datanode to namenode during
+   * distributed upgrade process.
+   */
+  rpc processUpgrade(ProcessUpgradeRequestProto) returns(ProcessUpgradeResponseProto);
+
+  /**
+   * Report corrupt blocks at the specified location
+   */
+  rpc reportBadBlocks(ReportBadBlocksRequestProto) returns(ReportBadBlocksResponseProto);
+
+  /**
+   * Commit block synchronization during lease recovery.
+   */
+  rpc commitBlockSynchronization(CommitBlockSynchronizationRequestProto)
+      returns(CommitBlockSynchronizationResponseProto);
+}

+ 27 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -41,6 +41,8 @@ Trunk (unreleased changes)
     (tucu)
 
   BUG FIXES
+    MAPREDUCE-3412. Fix 'ant docs'. (amarrk)
+
     MAPREDUCE-3346. [Rumen] LoggedTaskAttempt#getHostName() returns null. 
                     (amarrk)
 
@@ -106,6 +108,17 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3372. HADOOP_PREFIX cannot be overridden.
     (Bruno Mahé via tomwhite)
 
+    MAPREDUCE-3411. Performance Upgrade for jQuery (Jonathan Eagles via 
+    mahadev)
+
+    MAPREDUCE-3371. Review and improve the yarn-api javadocs. (Ravi Prakash
+    via mahadev)
+
+    MAPREDUCE-3238. Small cleanup in SchedulerApp. (Todd Lipcon via mahadev)
+
+    MAPREDUCE-3413. RM web ui applications not sorted in any order by default.
+    (Jonathan Eagles via mahadev)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -159,6 +172,20 @@ Release 0.23.1 - Unreleased
 
     MAPREDUCE-3444. trunk/0.23 builds broken (Hitesh Shah via mahadev)
 
+    MAPREDUCE-3454. [Gridmix] TestDistCacheEmulation is broken (Hitesh Shah
+    via mahadev)
+
+    MAPREDUCE-3408. yarn-daemon.sh unconditionnaly sets yarn.root.logger 
+    (Bruno Mahe via mahadev)
+
+    MAPREDUCE-3329. Fixed CapacityScheduler to ensure maximum-capacity cannot
+    be lesser than capacity for any queue. (acmurthy)
+
+    MAPREDUCE-3464. mapreduce jsp pages missing DOCTYPE. (Dave Vronay via mattf)
+
+    MAPREDUCE-3265. Removed debug logs during job submission to LOG.debug to
+    cut down noise. (acmurthy) 
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 0 - 4
hadoop-mapreduce-project/build.xml

@@ -932,10 +932,6 @@
     <style basedir="${mapred.src.dir}" destdir="${build.docs}"
            includes="mapred-default.xml" style="conf/configuration.xsl"/>
     <antcall target="changes-to-html"/>
-    <subant target="docs">
-       <property name="build.docs" value="${build.docs}"/>
-       <fileset file="${contrib.dir}/build.xml"/>
-    </subant> 
   </target>
 
   <target name="javadoc-dev" depends="compile, ivy-retrieve-javadoc" description="Generate javadoc for hadoop developers">

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppView.java

@@ -51,6 +51,8 @@ public class AppView extends TwoColumnLayout {
 
   private String jobsTableInit() {
     return tableInit().
+        // Sort by id upon page load
+        append(", aaSorting: [[0, 'asc']]").
         append(",aoColumns:[{sType:'title-numeric'},").
         append("null,null,{sType:'title-numeric', bSearchable:false},null,").
         append("null,{sType:'title-numeric',bSearchable:false}, null, null]}").

+ 4 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java

@@ -119,6 +119,9 @@ public class TaskPage extends AppView {
   }
 
   private String attemptsTableInit() {
-    return tableInit().append("}").toString();
+    return tableInit().
+        // Sort by id upon page load
+        append(", aaSorting: [[0, 'asc']]").
+        append("}").toString();
   }
 }

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java

@@ -38,6 +38,8 @@ public class TasksPage extends AppView {
 
   private String tasksTableInit() {
     return tableInit().
+        // Sort by id upon page load
+        append(", aaSorting: [[0, 'asc']]").
         append(",aoColumns:[{sType:'title-numeric'},{sType:'title-numeric',").
         append("bSearchable:false},null,{sType:'title-numeric'},").
         append("{sType:'title-numeric'},{sType:'title-numeric'}]}").toString();

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java

@@ -72,7 +72,7 @@ public class TaskLog {
     if (!LOG_DIR.exists()) {
       boolean b = LOG_DIR.mkdirs();
       if (!b) {
-        LOG.warn("mkdirs failed. Ignoring.");
+        LOG.debug("mkdirs failed. Ignoring.");
       }
     }
   }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java

@@ -108,7 +108,7 @@ public class Cluster {
             break;
           }
           else {
-            LOG.info("Cannot pick " + provider.getClass().getName()
+            LOG.debug("Cannot pick " + provider.getClass().getName()
                 + " as the ClientProtocolProvider - returned null protocol");
           }
         } 

+ 6 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java

@@ -296,8 +296,12 @@ public class HsTaskPage extends HsView {
     } else { //MAP
       b.append(", 5");
     }
-    b.append(" ] }");
-    b.append("]}");
+    b.append(" ] }]");
+
+    // Sort by id upon page load
+    b.append(", aaSorting: [[0, 'asc']]");
+
+    b.append("}");
     return b.toString();
   }
   

+ 6 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java

@@ -74,8 +74,12 @@ public class HsTasksPage extends HsView {
     } else { //MAP
       b.append(", 7");
     }
-    b.append(" ] }");
-    b.append("]}");
+    b.append(" ] }]");
+
+    // Sort by id upon page load
+    b.append(", aaSorting: [[0, 'asc']]");
+
+    b.append("}");
     return b.toString();
   }
   

+ 9 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java

@@ -84,11 +84,17 @@ public class HsView extends TwoColumnLayout {
    */
   private String jobsTableInit() {
     return tableInit().
-        append(",aoColumnDefs:[").
-        append("{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 6 ] }").
-        append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 7 ] }").
+        // Sort by id upon page load
+        append(", aaSorting: [[2, 'asc']]").
+        append(", aoColumnDefs:[").
+        // Maps Total
+        append("{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 7 ] }").
+        // Maps Completed
         append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 8 ] }").
+        // Reduces Total
         append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 9 ] }").
+        // Reduces Completed
+        append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 10 ] }").
         append("]}").
         toString();
   }

+ 2 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java

@@ -30,12 +30,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
-import org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SecurityInfo;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.YarnException;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 
 public class ClientCache {
@@ -79,9 +76,9 @@ public class ClientCache {
     if (StringUtils.isEmpty(serviceAddr)) {
       return null;
     }
-    LOG.info("Connecting to HistoryServer at: " + serviceAddr);
+    LOG.debug("Connecting to HistoryServer at: " + serviceAddr);
     final YarnRPC rpc = YarnRPC.create(conf);
-    LOG.info("Connected to HistoryServer at: " + serviceAddr);
+    LOG.debug("Connected to HistoryServer at: " + serviceAddr);
     UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
     return currentUser.doAs(new PrivilegedAction<MRClientProtocol>() {
       @Override

+ 8 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -143,7 +143,7 @@ public class ClientServiceDelegate {
         || YarnApplicationState.RUNNING == application
             .getYarnApplicationState()) {
       if (application == null) {
-        LOG.info("Could not get Job info from RM for job " + jobId
+        LOG.debug("Could not get Job info from RM for job " + jobId
             + ". Redirecting to job history server.");
         return checkAndGetHSProxy(null, JobState.NEW);
       }
@@ -169,8 +169,8 @@ public class ClientServiceDelegate {
               + ":" + addr.getPort()));
           UserGroupInformation.getCurrentUser().addToken(clientToken);
         }
-        LOG.info("Tracking Url of JOB is " + application.getTrackingUrl());
-        LOG.info("Connecting to " + serviceAddr);
+        LOG.info("The url to track the job: " + application.getTrackingUrl());
+        LOG.debug("Connecting to " + serviceAddr);
         realProxy = instantiateAMProxy(serviceAddr);
         return realProxy;
       } catch (IOException e) {
@@ -187,7 +187,7 @@ public class ClientServiceDelegate {
         }
         application = rm.getApplicationReport(appId);
         if (application == null) {
-          LOG.info("Could not get Job info from RM for job " + jobId
+          LOG.debug("Could not get Job info from RM for job " + jobId
               + ". Redirecting to job history server.");
           return checkAndGetHSProxy(null, JobState.RUNNING);
         }
@@ -281,16 +281,13 @@ public class ClientServiceDelegate {
           LOG.debug("Tracing remote error ", e.getTargetException());
           throw (YarnRemoteException) e.getTargetException();
         }
-        LOG.info("Failed to contact AM/History for job " + jobId + 
-            " retrying..");
-        LOG.debug("Failed exception on AM/History contact", 
-            e.getTargetException());
+        LOG.debug("Failed to contact AM/History for job " + jobId + 
+            " retrying..", e.getTargetException());
         // Force reconnection by setting the proxy to null.
         realProxy = null;
       } catch (Exception e) {
-        LOG.info("Failed to contact AM/History for job " + jobId
-            + "  Will retry..");
-        LOG.debug("Failing to contact application master", e);
+        LOG.debug("Failed to contact AM/History for job " + jobId
+            + "  Will retry..", e);
         // Force reconnection by setting the proxy to null.
         realProxy = null;
       }

+ 18 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java

@@ -25,7 +25,6 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -40,11 +39,9 @@ import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SecurityInfo;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
@@ -56,6 +53,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -67,13 +65,13 @@ import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo;
 
 
 // TODO: This should be part of something like yarn-client.
 public class ResourceMgrDelegate {
   private static final Log LOG = LogFactory.getLog(ResourceMgrDelegate.class);
       
+  private final String rmAddress;
   private YarnConfiguration conf;
   ClientRMProtocol applicationsManager;
   private ApplicationId applicationId;
@@ -92,21 +90,25 @@ public class ResourceMgrDelegate {
             YarnConfiguration.DEFAULT_RM_ADDRESS),
             YarnConfiguration.DEFAULT_RM_PORT,
             YarnConfiguration.RM_ADDRESS);
-    LOG.info("Connecting to ResourceManager at " + rmAddress);
+    this.rmAddress = rmAddress.toString();
+    LOG.debug("Connecting to ResourceManager at " + rmAddress);
     applicationsManager =
         (ClientRMProtocol) rpc.getProxy(ClientRMProtocol.class,
             rmAddress, this.conf);
-    LOG.info("Connected to ResourceManager at " + rmAddress);
+    LOG.debug("Connected to ResourceManager at " + rmAddress);
   }
   
   /**
    * Used for injecting applicationsManager, mostly for testing.
    * @param conf the configuration object
-   * @param applicationsManager the handle to talk the resource managers {@link ClientRMProtocol}.
+   * @param applicationsManager the handle to talk the resource managers 
+   *                            {@link ClientRMProtocol}.
    */
-  public ResourceMgrDelegate(YarnConfiguration conf, ClientRMProtocol applicationsManager) {
+  public ResourceMgrDelegate(YarnConfiguration conf, 
+      ClientRMProtocol applicationsManager) {
     this.conf = conf;
     this.applicationsManager = applicationsManager;
+    this.rmAddress = applicationsManager.toString();
   }
   
   public void cancelDelegationToken(Token<DelegationTokenIdentifier> arg0)
@@ -295,18 +297,22 @@ public class ResourceMgrDelegate {
   }
   
   
-  public ApplicationId submitApplication(ApplicationSubmissionContext appContext) 
+  public ApplicationId submitApplication(
+      ApplicationSubmissionContext appContext) 
   throws IOException {
     appContext.setApplicationId(applicationId);
-    SubmitApplicationRequest request = recordFactory.newRecordInstance(SubmitApplicationRequest.class);
+    SubmitApplicationRequest request = 
+        recordFactory.newRecordInstance(SubmitApplicationRequest.class);
     request.setApplicationSubmissionContext(appContext);
     applicationsManager.submitApplication(request);
-    LOG.info("Submitted application " + applicationId + " to ResourceManager");
+    LOG.info("Submitted application " + applicationId + " to ResourceManager" +
+    		" at " + rmAddress);
     return applicationId;
   }
   
   public void killApplication(ApplicationId applicationId) throws IOException {
-    KillApplicationRequest request = recordFactory.newRecordInstance(KillApplicationRequest.class);
+    KillApplicationRequest request = 
+        recordFactory.newRecordInstance(KillApplicationRequest.class);
     request.setApplicationId(applicationId);
     applicationsManager.forceKillApplication(request);
     LOG.info("Killing application " + applicationId);

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java

@@ -276,7 +276,7 @@ public class YARNRunner implements ClientProtocol {
     Resource capability = recordFactory.newRecordInstance(Resource.class);
     capability.setMemory(conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
         MRJobConfig.DEFAULT_MR_AM_VMEM_MB));
-    LOG.info("AppMaster capability = " + capability);
+    LOG.debug("AppMaster capability = " + capability);
 
     // Setup LocalResources
     Map<String, LocalResource> localResources =
@@ -352,7 +352,7 @@ public class YARNRunner implements ClientProtocol {
     }
     vargsFinal.add(mergedCommand.toString());
 
-    LOG.info("Command to launch container for ApplicationMaster is : "
+    LOG.debug("Command to launch container for ApplicationMaster is : "
         + mergedCommand);
 
     // Setup the CLASSPATH in environment

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/bin/yarn-daemon.sh

@@ -87,8 +87,8 @@ fi
 
 # some variables
 export YARN_LOGFILE=yarn-$YARN_IDENT_STRING-$command-$HOSTNAME.log
-export YARN_ROOT_LOGGER="INFO,DRFA"
-export YARN_JHS_LOGGER="INFO,JSA"
+export YARN_ROOT_LOGGER=${YARN_ROOT_LOGGER:-INFO,DRFA}
+export YARN_JHS_LOGGER=${YARN_JHS_LOGGER:-INFO,JSA}
 log=$YARN_LOG_DIR/yarn-$YARN_IDENT_STRING-$command-$HOSTNAME.out
 pid=$YARN_PID_DIR/yarn-$YARN_IDENT_STRING-$command.pid
 

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/AMRMProtocol.java

@@ -91,8 +91,8 @@ public interface AMRMProtocol {
    * 
    * <p>This also doubles up as a <em>heartbeat</em> to let the 
    * <code>ResourceManager</code> know that the <code>ApplicationMaster</code>
-   * is alive. Thus, applications should use periodically make this call to 
-   * be kept alive.</p>
+   * is alive. Thus, applications should periodically make this call to be kept
+   * alive. The frequency depends on ??</p>
    * 
    * <p>The <code>ResourceManager</code> responds with list of allocated 
    * {@link Container}, status of completed containers and headroom information 

+ 3 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java

@@ -68,7 +68,8 @@ public interface ClientRMProtocol {
    * {@link GetNewApplicationResponse}.</p>
    *
    * @param request request to get a new <code>ApplicationId</code>
-   * @return new <code>ApplicationId</code> to be used to submit an application
+   * @return response containing the new <code>ApplicationId</code> to be used
+   * to submit an application
    * @throws YarnRemoteException
    * @see #submitApplication(SubmitApplicationRequest)
    */
@@ -216,7 +217,7 @@ public interface ClientRMProtocol {
   
   /**
    * <p>The interface used by clients to get information about <em>queue 
-   * acls</em> for <em>current users</em> from the <code>ResourceManager</code>.
+   * acls</em> for <em>current user</em> from the <code>ResourceManager</code>.
    * </p>
    * 
    * <p>The <code>ResourceManager</code> responds with queue acls for all

+ 5 - 4
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManager.java

@@ -79,7 +79,7 @@ public interface ContainerManager {
    * to <em>stop</em> a {@link Container} allocated to it using this interface.
    * </p>
    * 
-   * <p>The <code>ApplicationMaster</code></p> sends a 
+   * <p>The <code>ApplicationMaster</code> sends a
    * {@link StopContainerRequest} which includes the {@link ContainerId} of the
    * container to be stopped.</p>
    * 
@@ -105,8 +105,8 @@ public interface ContainerManager {
    * current status of a <code>Container</code> from the 
    * <code>NodeManager</code>.</p>
    * 
-   * <p>The <code>ApplicationMaster</code></p> sends a 
-   * {@link GetContainerStatusRequest} which includes the {@link ContainerId} of 
+   * <p>The <code>ApplicationMaster</code> sends a
+   * {@link GetContainerStatusRequest} which includes the {@link ContainerId} of
    * the container whose status is needed.</p>
    *
    *<p>The <code>NodeManager</code> responds with 
@@ -115,7 +115,8 @@ public interface ContainerManager {
    *
    * @param request request to get <code>ContainerStatus</code> of a container
    *                with the specified <code>ContainerId</code>
-   * @return <code>ContainerStatus</code> of the container
+   * @return response containing the <code>ContainerStatus</code> of the
+   * container
    * @throws YarnRemoteException
    */
   @Public

+ 5 - 6
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java

@@ -50,7 +50,6 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
  *     <li>
  *       A list of unused {@link Container} which are being returned. 
  *     </li>
- *     <li></li>
  *   </ul>
  * </p>
  * 
@@ -81,7 +80,7 @@ public interface AllocateRequest {
   void setApplicationAttemptId(ApplicationAttemptId applicationAttemptId);
 
   /**
-   * Get the <em>response id</em>.
+   * Get the <em>response id</em> used to track duplicate responses.
    * @return <em>response id</em>
    */
   @Public
@@ -89,7 +88,7 @@ public interface AllocateRequest {
   int getResponseId();
 
   /**
-   * Set the <em>response id</em>
+   * Set the <em>response id</em> used to track duplicate responses.
    * @param id <em>response id</em>
    */
   @Public
@@ -113,7 +112,7 @@ public interface AllocateRequest {
   void setProgress(float progress);
 
   /**
-   * Get the list of <code>ResourceRequest</code> to upate the 
+   * Get the list of <code>ResourceRequest</code> to update the 
    * <code>ResourceManager</code> about the application's resource requirements.
    * @return the list of <code>ResourceRequest</code>
    */
@@ -130,9 +129,9 @@ public interface AllocateRequest {
   int getAskCount();
   
   /**
-   * Add list of <code>ResourceRequest</code> to upate the 
+   * Add list of <code>ResourceRequest</code> to update the 
    * <code>ResourceManager</code> about the application's resource requirements.
-   * @param resourceRequest list of <code>ResourceRequest</code> to upate the 
+   * @param resourceRequest list of <code>ResourceRequest</code> to update the 
    *                        <code>ResourceManager</code> about the application's 
    *                        resource requirements
    */

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.api.records.Container;
  *   <ul>
  *     <li>Response ID to track duplicate responses.</li>
  *     <li>
- *       A reboot flag to let the <code>ApplicationMaster</code> that its 
+ *       A reboot flag to let the <code>ApplicationMaster</code> know that its 
  *       horribly out of sync and needs to reboot.</li>
  *     <li>A list of newly allocated {@link Container}.</li>
  *     <li>A list of completed {@link Container}.</li>

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAllApplicationsRequest.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
  * <p>The request from clients to get a report of all Applications
  * in the cluster from the <code>ResourceManager</code>.</p>
  *
+ * <p>Currently, this is empty.</p>
+ * 
  * @see ClientRMProtocol#getAllApplications(GetAllApplicationsRequest)
  */
 @Public

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsRequest.java

@@ -25,6 +25,8 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
 /**
  * <p>The request sent by clients to get cluster metrics from the 
  * <code>ResourceManager</code>.</p>
+ * 
+ * <p>Currently, this is empty.</p>
  *
  * @see ClientRMProtocol#getClusterMetrics(GetClusterMetricsRequest)
  */

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesRequest.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
  * <p>The request from clients to get a report of all nodes
  * in the cluster from the <code>ResourceManager</code>.</p>
  *
+ * <p>Currently, this is empty.</p>
+ *
  * @see ClientRMProtocol#getClusterNodes(GetClusterNodesRequest) 
  */
 @Public

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodesResponse.java

@@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 
 /**
  * <p>The response sent by the <code>ResourceManager</code> to a client
- * requesting an {@link NodeReport} for all nodes.</p>
+ * requesting a {@link NodeReport} for all nodes.</p>
  * 
  * <p>The <code>NodeReport</code> contains per-node information such as 
  * available resources, number of containers, tracking url, rack name, health

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusResponse.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 
 /**
  * <p>The response sent by the <code>NodeManager</code> to the 
- * <code>ApplicationMaster</code> when asked to obtainer <em>status</em> 
+ * <code>ApplicationMaster</code> when asked to obtain the <em>status</em> 
  * of a container.</p>
  * 
  * @see ContainerManager#getContainerStatus(GetContainerStatusRequest)

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationRequest.java

@@ -27,6 +27,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
  * <p>The request sent by clients to get a new {@link ApplicationId} for
  * submitting an application.</p>
  * 
+ * <p>Currently, this is empty.</p>
+ * 
  * @see ClientRMProtocol#getNewApplication(GetNewApplicationRequest)
  */
 @Public

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationResponse.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 
 /**
  * <p>The response sent by the <code>ResourceManager</code> to the client for 
- * a request to a new {@link ApplicationId} for submitting applications.</p>
+ * a request to get a new {@link ApplicationId} for submitting applications.</p>
  * 
  * @see ClientRMProtocol#getNewApplication(GetNewApplicationRequest)
  */

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueUserAclsInfoRequest.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
  * <p>The request sent by clients to the <code>ResourceManager</code> to 
  * get queue acls for the <em>current user</em>.</p>
  *
+ * <p>Currently, this is empty.</p>
+ * 
  * @see ClientRMProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest)
  */
 @Public

+ 42 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerResponse.java

@@ -35,11 +35,53 @@ import org.apache.hadoop.yarn.api.ContainerManager;
 @Public
 @Stable
 public interface StartContainerResponse {
+  /**
+   * <p>Get the responses from all auxiliary services running on the 
+   * <code>NodeManager</code>.</p>
+   * <p>The responses are returned as a Map between the auxiliary service names
+   * and their corresponding opaque blob <code>ByteBuffer</code>s</p> 
+   * @return a Map between the auxiliary service names and their outputs
+   */
   Map<String, ByteBuffer> getAllServiceResponse();
+
+  /**
+   * Get the response from a single auxiliary service running on the
+   * <code>NodeManager</code>
+   * 
+   * @param key The auxiliary service name whose response is desired.
+   * @return The opaque blob <code>ByteBuffer</code> returned by the auxiliary
+   * service.
+   */
   ByteBuffer getServiceResponse(String key);
 
+  /**
+   * Add to the list of auxiliary services which have been started on the
+   * <code>NodeManager</code>. This is done only once when the
+   * <code>NodeManager</code> starts up
+   * @param serviceResponse A map from auxiliary service names to the opaque
+   * blob <code>ByteBuffer</code>s for that auxiliary service
+   */
   void addAllServiceResponse(Map<String, ByteBuffer> serviceResponse);
+
+  /**
+   * Add to the list of auxiliary services which have been started on the
+   * <code>NodeManager</code>. This is done only once when the
+   * <code>NodeManager</code> starts up
+   * 
+   * @param key The auxiliary service name
+   * @param value The opaque blob <code>ByteBuffer</code> managed by the
+   * auxiliary service
+   */
   void setServiceResponse(String key, ByteBuffer value);
+
+  /**
+   * Remove a single auxiliary service from the StartContainerResponse object
+   * @param key The auxiliary service to remove
+   */
   void removeServiceResponse(String key);
+  
+  /**
+   * Remove all the auxiliary services from the StartContainerResponse object
+   */
   void clearServiceResponse();
 }

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerResponse.java

@@ -27,6 +27,8 @@ import org.apache.hadoop.yarn.api.ContainerManager;
  * <code>ApplicationMaster</code> when asked to <em>stop</em> an
  * allocated container.</p>
  * 
+ * <p>Currently, this is empty.</p>
+ * 
  * @see ContainerManager#stopContainer(StopContainerRequest)
  */
 @Public

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SubmitApplicationResponse.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
  * <p>The response sent by the <code>ResourceManager</code> to a client on
  * application submission.</p>
  * 
+ * <p>Currently, this is empty.</p>
+ * 
  * @see ClientRMProtocol#submitApplication(SubmitApplicationRequest)
  */
 @Public

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMResponse.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
  *   <ul>
  *     <li>Response ID to track duplicate responses.</li>
  *     <li>
- *       A reboot flag to let the <code>ApplicationMaster</code> that its 
+ *       A reboot flag to let the <code>ApplicationMaster</code> know that its 
  *       horribly out of sync and needs to reboot.</li>
  *     <li>A list of newly allocated {@link Container}.</li>
  *     <li>A list of completed {@link Container}.</li>
@@ -100,7 +100,7 @@ public interface AMResponse {
   /**
    * Get the <em>available headroom</em> for resources in the cluster for the 
    * application.
-   * @return limit available headroom for resources in the cluster for the 
+   * @return limit of available headroom for resources in the cluster for the 
    * application
    */
   @Public

+ 2 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
  *     <li>Applications user.</li>
  *     <li>Application queue.</li>
  *     <li>Application name.</li>
- *     <li>Host on which the <code>ApplicationMaster</code>is running.</li>
+ *     <li>Host on which the <code>ApplicationMaster</code> is running.</li>
  *     <li>RPC port of the <code>ApplicationMaster</code>.</li>
  *     <li>Tracking URL.</li>
  *     <li>{@link YarnApplicationState} of the application.</li>
@@ -215,6 +215,7 @@ public interface ApplicationReport {
 
   /**
    * Get the <em>final finish status</em> of the application.
+   * @return <em>final finish status</em> of the application
    */
   @Public
   @Stable

+ 29 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java

@@ -33,22 +33,43 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Public
 @Stable
 public interface ApplicationResourceUsageReport {
+
+  /**
+   * Get the number of used containers
+   * @return the number of used containers
+   */
   @Public
   @Stable
   int getNumUsedContainers();
 
+  /**
+   * Set the number of used containers
+   * @param num_containers the number of used containers
+   */
   @Private
   @Unstable
   void setNumUsedContainers(int num_containers);
 
+  /**
+   * Get the number of reserved containers
+   * @return the number of reserved containers
+   */
   @Public
   @Stable
   int getNumReservedContainers();
 
+  /**
+   * Set the number of reserved containers
+   * @param num_reserved_containers the number of reserved containers
+   */
   @Private
   @Unstable
   void setNumReservedContainers(int num_reserved_containers);
 
+  /**
+   * Get the used <code>Resource</code>
+   * @return the used <code>Resource</code>
+   */
   @Public
   @Stable
   Resource getUsedResources();
@@ -57,6 +78,10 @@ public interface ApplicationResourceUsageReport {
   @Unstable
   void setUsedResources(Resource resources);
 
+  /**
+   * Get the reserved <code>Resource</code>
+   * @return the reserved <code>Resource</code>
+   */
   @Public
   @Stable
   Resource getReservedResources();
@@ -65,6 +90,10 @@ public interface ApplicationResourceUsageReport {
   @Unstable
   void setReservedResources(Resource reserved_resources);
 
+  /**
+   * Get the needed <code>Resource</code>
+   * @return the needed <code>Resource</code>
+   */
   @Public
   @Stable
   Resource getNeededResources();

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 
 /**
- * <p><code>ApplicationSubmissionContext</code> represents the all of the 
+ * <p><code>ApplicationSubmissionContext</code> represents all of the
  * information needed by the <code>ResourceManager</code> to launch 
  * the <code>ApplicationMaster</code> for an application.</p>
  * 

+ 1 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java

@@ -38,8 +38,7 @@ import org.apache.hadoop.yarn.api.ContainerManager;
  *   <ul>
  *     <li>{@link ContainerId} for the container, which is globally unique.</li>
  *     <li>
- *       {@link NodeId} of the node on which identifies the node on which it
- *       is allocated.
+ *       {@link NodeId} of the node on which it is allocated.
  *     </li>
  *     <li>HTTP uri of the node.</li>
  *     <li>{@link Resource} allocated to the container.</li>

+ 12 - 7
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ContainerManager;
 
 /**
- * <p><code>ContainerLaunchContext</code> represents the all of the information
+ * <p><code>ContainerLaunchContext</code> represents all of the information
  * needed by the <code>NodeManager</code> to launch a container.</p>
  * 
  * <p>It includes details such as:
@@ -127,7 +127,8 @@ public interface ContainerLaunchContext {
   Map<String, LocalResource> getLocalResources();
   
   /**
-   * Set <code>LocalResource</code> required by the container.
+   * Set <code>LocalResource</code> required by the container. All pre-existing
+   * Map entries are cleared before adding the new Map
    * @param localResources <code>LocalResource</code> required by the container
    */
   @Public
@@ -143,7 +144,8 @@ public interface ContainerLaunchContext {
   Map<String, ByteBuffer> getServiceData();
   
   /**
-   * Set application-specific binary <em>service data</em>.
+   * Set application-specific binary <em>service data</em>. All pre-existing Map
+   * entries are preserved.
    * @param serviceData application-specific binary <em>service data</em>
    */
   @Public
@@ -159,7 +161,8 @@ public interface ContainerLaunchContext {
   Map<String, String> getEnvironment();
     
   /**
-   * Add <em>environment variables</em> for the container.
+   * Add <em>environment variables</em> for the container. All pre-existing Map
+   * entries are cleared before adding the new Map
    * @param environment <em>environment variables</em> for the container
    */
   @Public
@@ -175,7 +178,8 @@ public interface ContainerLaunchContext {
   List<String> getCommands();
   
   /**
-   * Add the list of <em>commands</em> for launching the container.
+   * Add the list of <em>commands</em> for launching the container. All
+   * pre-existing List entries are cleared before adding the new List
    * @param commands the list of <em>commands</em> for launching the container
    */
   @Public
@@ -191,8 +195,9 @@ public interface ContainerLaunchContext {
   public Map<ApplicationAccessType, String> getApplicationACLs();
 
   /**
-   * Set the <code>ApplicationACL</code>s for the application. 
-   * @param acls
+   * Set the <code>ApplicationACL</code>s for the application. All pre-existing
+   * Map entries are cleared before adding the new Map
+   * @param acls <code>ApplicationACL</code>s for the application
    */
   @Public
   @Stable

+ 0 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeHealthStatus.java

@@ -35,8 +35,6 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
  *     </li>
  *     <li>The previous time at which the health status was reported.</li>
  *     <li>A diagnostic report on the health status.</li>
- *     <li></li>
- *     <li></li>
  *   </ul>
  * </p>
  * 

+ 13 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java

@@ -18,10 +18,23 @@
 
 package org.apache.hadoop.yarn.api.records;
 
+/**
+ * The priority assigned to a ResourceRequest or Application or Container 
+ * allocation 
+ *
+ */
 public interface Priority extends Comparable<Priority> {
   
+  /**
+   * Get the assigned priority
+   * @return the assigned priority
+   */
   public abstract int getPriority();
   
+  /**
+   * Set the assigned priority
+   * @param priority the assigned priority
+   */
   public abstract void setPriority(int priority);
   
 }

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 
 /**
- * <p>QueueInfo</p> is a report of the runtime information of the queue.</p>
+ * <p>QueueInfo is a report of the runtime information of the queue.</p>
  * 
  * <p>It includes information such as:
  *   <ul>

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -93,7 +93,7 @@ import org.apache.hadoop.yarn.util.Records;
  * to inform the <code>ResourceManager</code> that it is up and alive. The {@link AMRMProtocol#allocate} to the 
  * <code>ResourceManager</code> from the <code>ApplicationMaster</code> acts as a heartbeat.
  * 
- * <p> For the actual handling of the job, the <code>ApplicationMaster</code> has to request for the 
+ * <p> For the actual handling of the job, the <code>ApplicationMaster</code> has to request the 
  * <code>ResourceManager</code> via {@link AllocateRequest} for the required no. of containers using {@link ResourceRequest}
  * with the necessary resource specifications such as node location, computational (memory/disk/cpu) resource requirements.
  * The <code>ResourceManager</code> responds with an {@link AllocateResponse} that informs the <code>ApplicationMaster</code> 

+ 5 - 5
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java

@@ -37,12 +37,12 @@ import org.apache.hadoop.yarn.factory.providers.RpcFactoryProvider;
  */
 public class HadoopYarnProtoRPC extends YarnRPC {
 
-  private static final Log LOG = LogFactory.getLog(HadoopYarnRPC.class);
+  private static final Log LOG = LogFactory.getLog(HadoopYarnProtoRPC.class);
 
   @Override
   public Object getProxy(Class protocol, InetSocketAddress addr,
       Configuration conf) {
-    LOG.info("Creating a HadoopYarnProtoRpc proxy for protocol " + protocol);
+    LOG.debug("Creating a HadoopYarnProtoRpc proxy for protocol " + protocol);
     return RpcFactoryProvider.getClientFactory(conf).getClient(protocol, 1,
         addr, conf);
   }
@@ -57,11 +57,11 @@ public class HadoopYarnProtoRPC extends YarnRPC {
       InetSocketAddress addr, Configuration conf,
       SecretManager<? extends TokenIdentifier> secretManager,
       int numHandlers) {
-    LOG.info("Creating a HadoopYarnProtoRpc server for protocol " + protocol + 
+    LOG.debug("Creating a HadoopYarnProtoRpc server for protocol " + protocol + 
         " with " + numHandlers + " handlers");
     
-    return RpcFactoryProvider.getServerFactory(conf).getServer(protocol, instance, 
-          addr, conf, secretManager, numHandlers);
+    return RpcFactoryProvider.getServerFactory(conf).getServer(protocol, 
+        instance, addr, conf, secretManager, numHandlers);
 
   }
 

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java

@@ -45,7 +45,7 @@ public class HadoopYarnRPC extends YarnRPC {
   @Override
   public Object getProxy(Class protocol, InetSocketAddress addr,
       Configuration conf) {
-    LOG.info("Creating a HadoopYarnRpc proxy for protocol " + protocol);
+    LOG.debug("Creating a HadoopYarnRpc proxy for protocol " + protocol);
     RPC.setProtocolEngine(conf, protocol, AvroSpecificRpcEngine.class);
     try {
       return RPC.getProxy(protocol, 1, addr, conf);
@@ -64,7 +64,7 @@ public class HadoopYarnRPC extends YarnRPC {
       InetSocketAddress addr, Configuration conf,
       SecretManager<? extends TokenIdentifier> secretManager,
       int numHandlers) {
-    LOG.info("Creating a HadoopYarnRpc server for protocol " + protocol + 
+    LOG.debug("Creating a HadoopYarnRpc server for protocol " + protocol + 
         " with " + numHandlers + " handlers");
     RPC.setProtocolEngine(conf, protocol, AvroSpecificRpcEngine.class);
     final RPC.Server hadoopServer;

+ 2 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java

@@ -46,7 +46,8 @@ public abstract class YarnRPC {
       int numHandlers);
 
   public static YarnRPC create(Configuration conf) {
-    LOG.info("Creating YarnRPC for " + conf.get(YarnConfiguration.IPC_RPC_IMPL));
+    LOG.debug("Creating YarnRPC for " + 
+        conf.get(YarnConfiguration.IPC_RPC_IMPL));
     String clazzName = conf.get(YarnConfiguration.IPC_RPC_IMPL);
     if (clazzName == null) {
       clazzName = YarnConfiguration.DEFAULT_IPC_RPC_IMPL;

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ApplicationTokenSelector.java

@@ -39,9 +39,9 @@ public class ApplicationTokenSelector implements
     if (service == null) {
       return null;
     }
-    LOG.info("Looking for a token with service " + service.toString());
+    LOG.debug("Looking for a token with service " + service.toString());
     for (Token<? extends TokenIdentifier> token : tokens) {
-      LOG.info("Token kind is " + token.getKind().toString()
+      LOG.debug("Token kind is " + token.getKind().toString()
           + " and the token's service name is " + token.getService());
       if (ApplicationTokenIdentifier.KIND_NAME.equals(token.getKind())
           && service.equals(token.getService())) {

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientTokenSelector.java

@@ -39,9 +39,9 @@ public class ClientTokenSelector implements
     if (service == null) {
       return null;
     }
-    LOG.info("Looking for a token with service " + service.toString());
+    LOG.debug("Looking for a token with service " + service.toString());
     for (Token<? extends TokenIdentifier> token : tokens) {
-      LOG.info("Token kind is " + token.getKind().toString()
+      LOG.debug("Token kind is " + token.getKind().toString()
           + " and the token's service name is " + token.getService());
       if (ClientTokenIdentifier.KIND_NAME.equals(token.getKind())
           && service.equals(token.getService())) {

+ 4 - 4
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java

@@ -79,11 +79,11 @@ public class JQueryUI extends HtmlBlock {
   @Override
   protected void render(Block html) {
     html.
-      link(join("https://ajax.googleapis.com/ajax/libs/jqueryui/1.8.9/themes/",
+      link(join("https://ajax.googleapis.com/ajax/libs/jqueryui/1.8.16/themes/",
                 getTheme(), "/jquery-ui.css")).
       link(root_url("static/dt-1.7.5/css/jui-dt.css")).
-      script("https://ajax.googleapis.com/ajax/libs/jquery/1.4.4/jquery.min.js").
-      script("https://ajax.googleapis.com/ajax/libs/jqueryui/1.8.9/jquery-ui.min.js").
+      script("https://ajax.googleapis.com/ajax/libs/jquery/1.6.4/jquery.min.js").
+      script("https://ajax.googleapis.com/ajax/libs/jqueryui/1.8.16/jquery-ui.min.js").
       script(root_url("static/dt-1.7.5/js/jquery.dataTables.min.js")).
       script(root_url("static/yarn.dt.plugins.js")).
       script(root_url("static/themeswitcher.js")).
@@ -224,7 +224,7 @@ public class JQueryUI extends HtmlBlock {
   }
 
   public static StringBuilder tableInit() {
-    return new StringBuilder("{bJQueryUI:true, aaSorting:[], ").
+    return new StringBuilder("{bJQueryUI:true, ").
         append("sPaginationType: 'full_numbers', iDisplayLength:20, ").
         append("aLengthMenu:[20, 40, 60, 80, 100]");
   }

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java

@@ -68,7 +68,7 @@ public class AuxServices extends AbstractService
   /**
    * @return the meta data for all registered services, that have been started.
    * If a service has not been started no metadata will be available. The key
-   * the the name of the service as defined in the configuration.
+   * is the name of the service as defined in the configuration.
    */
   public Map<String, ByteBuffer> getMeta() {
     Map<String, ByteBuffer> metaClone = new HashMap<String, ByteBuffer>(

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.java

@@ -51,6 +51,8 @@ public class AllApplicationsPage extends NMView {
 
   private String appsTableInit() {
     return tableInit().
+        // Sort by id upon page load
+        append(", aaSorting: [[0, 'asc']]").
         // applicationid, applicationstate
         append(", aoColumns:[null, null]} ").toString();
   }

+ 29 - 32
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java

@@ -53,6 +53,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Multiset;
+
+/**
+ * Represents an Application from the viewpoint of the scheduler.
+ * Each running Application in the RM corresponds to one instance
+ * of this class.
+ */
 public class SchedulerApp {
 
   private static final Log LOG = LogFactory.getLog(SchedulerApp.class);
@@ -76,11 +84,16 @@ public class SchedulerApp {
   final Map<Priority, Map<NodeId, RMContainer>> reservedContainers = 
       new HashMap<Priority, Map<NodeId, RMContainer>>();
   
-  Map<Priority, Integer> schedulingOpportunities = 
-      new HashMap<Priority, Integer>();
+  /**
+   * Count how many times the application has been given an opportunity
+   * to schedule a task at each priority. Each time the scheduler
+   * asks the application for a task at this priority, it is incremented,
+   * and each time the application successfully schedules a task, it
+   * is reset to 0.
+   */
+  Multiset<Priority> schedulingOpportunities = HashMultiset.create();
   
-  Map<Priority, Integer> reReservations =
-      new HashMap<Priority, Integer>();
+  Multiset<Priority> reReservations = HashMultiset.create();
 
   Resource currentReservation = recordFactory
       .newRecordInstance(Resource.class);
@@ -282,49 +295,33 @@ public class SchedulerApp {
   }
 
   synchronized public void resetSchedulingOpportunities(Priority priority) {
-    this.schedulingOpportunities.put(priority, Integer.valueOf(0));
+    this.schedulingOpportunities.setCount(priority, 0);
   }
 
   synchronized public void addSchedulingOpportunity(Priority priority) {
-    Integer schedulingOpportunities = 
-        this.schedulingOpportunities.get(priority);
-    if (schedulingOpportunities == null) {
-      schedulingOpportunities = 0;
-    }
-    ++schedulingOpportunities;
-    this.schedulingOpportunities.put(priority, schedulingOpportunities);
+    this.schedulingOpportunities.setCount(priority,
+        schedulingOpportunities.count(priority) + 1);
   }
 
+  /**
+   * Return the number of times the application has been given an opportunity
+   * to schedule a task at the given priority since the last time it
+   * successfully did so.
+   */
   synchronized public int getSchedulingOpportunities(Priority priority) {
-    Integer schedulingOpportunities = 
-        this.schedulingOpportunities.get(priority);
-    if (schedulingOpportunities == null) {
-      schedulingOpportunities = 0;
-      this.schedulingOpportunities.put(priority, schedulingOpportunities);
-    }
-    return schedulingOpportunities;
+    return this.schedulingOpportunities.count(priority);
   }
 
   synchronized void resetReReservations(Priority priority) {
-    this.reReservations.put(priority, Integer.valueOf(0));
+    this.reReservations.setCount(priority, 0);
   }
 
   synchronized void addReReservation(Priority priority) {
-    Integer reReservations = this.reReservations.get(priority);
-    if (reReservations == null) {
-      reReservations = 0;
-    }
-    ++reReservations;
-    this.reReservations.put(priority, reReservations);
+    this.reReservations.add(priority);
   }
 
   synchronized public int getReReservations(Priority priority) {
-    Integer reReservations = this.reReservations.get(priority);
-    if (reReservations == null) {
-      reReservations = 0;
-      this.reReservations.put(priority, reReservations);
-    }
-    return reReservations;
+    return this.reReservations.count(priority);
   }
 
   public synchronized int getNumReservedContainers(Priority priority) {

+ 34 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java

@@ -0,0 +1,34 @@
+/**
+* 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.yarn.server.resourcemanager.scheduler.capacity;
+
+class CSQueueUtils {
+  
+  public static void checkMaxCapacity(String queueName, 
+      float capacity, float maximumCapacity) {
+    if (maximumCapacity != CapacitySchedulerConfiguration.UNDEFINED && 
+        maximumCapacity < capacity) {
+      throw new IllegalArgumentException(
+          "Illegal call to setMaxCapacity. " +
+          "Queue '" + queueName + "' has " +
+          "capacity (" + capacity + ") greater than " + 
+          "maximumCapacity (" + maximumCapacity + ")" );
+    }
+  }
+  
+}

+ 4 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java

@@ -160,6 +160,10 @@ public class CapacitySchedulerConfiguration extends Configuration {
     return maxCapacity;
   }
   
+  public void setMaximumCapacity(String queue, int maxCapacity) {
+    setInt(getQueuePrefix(queue) + MAXIMUM_CAPACITY, maxCapacity);
+  }
+  
   public int getUserLimit(String queue) {
     int userLimit = 
       getInt(getQueuePrefix(queue) + USER_LIMIT, DEFAULT_USER_LIMIT);

+ 12 - 6
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -211,16 +211,19 @@ public class LeafQueue implements CSQueue {
   
   private synchronized void setupQueueConfigs(
       float capacity, float absoluteCapacity, 
-      float maxCapacity, float absoluteMaxCapacity,
+      float maximumCapacity, float absoluteMaxCapacity,
       int userLimit, float userLimitFactor,
       int maxApplications, int maxApplicationsPerUser,
       int maxActiveApplications, int maxActiveApplicationsPerUser,
       QueueState state, Map<QueueACL, AccessControlList> acls)
   {
+    // Sanity check
+    CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
+
     this.capacity = capacity; 
     this.absoluteCapacity = parent.getAbsoluteCapacity() * capacity;
 
-    this.maximumCapacity = maxCapacity;
+    this.maximumCapacity = maximumCapacity;
     this.absoluteMaxCapacity = absoluteMaxCapacity;
 
     this.userLimit = userLimit;
@@ -236,9 +239,9 @@ public class LeafQueue implements CSQueue {
 
     this.acls = acls;
 
-    this.queueInfo.setCapacity(capacity);
-    this.queueInfo.setMaximumCapacity(maximumCapacity);
-    this.queueInfo.setQueueState(state);
+    this.queueInfo.setCapacity(this.capacity);
+    this.queueInfo.setMaximumCapacity(this.maximumCapacity);
+    this.queueInfo.setQueueState(this.state);
 
     StringBuilder aclsString = new StringBuilder();
     for (Map.Entry<QueueACL, AccessControlList> e : acls.entrySet()) {
@@ -250,7 +253,7 @@ public class LeafQueue implements CSQueue {
         " [= (float) configuredCapacity / 100 ]" + "\n" + 
         "asboluteCapacity = " + absoluteCapacity +
         " [= parentAbsoluteCapacity * capacity ]" + "\n" +
-        "maxCapacity = " + maxCapacity +
+        "maxCapacity = " + maximumCapacity +
         " [= configuredMaxCapacity ]" + "\n" +
         "absoluteMaxCapacity = " + absoluteMaxCapacity +
         " [= Float.MAX_VALUE if maximumCapacity undefined, " +
@@ -394,6 +397,9 @@ public class LeafQueue implements CSQueue {
    * @param maximumCapacity new max capacity
    */
   synchronized void setMaxCapacity(float maximumCapacity) {
+    // Sanity check
+    CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
+    
     this.maximumCapacity = maximumCapacity;
     this.absoluteMaxCapacity = 
       (maximumCapacity == CapacitySchedulerConfiguration.UNDEFINED) ? 

+ 9 - 3
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -153,6 +153,9 @@ public class ParentQueue implements CSQueue {
           float maximumCapacity, float absoluteMaxCapacity,
           QueueState state, Map<QueueACL, AccessControlList> acls
   ) {
+    // Sanity check
+    CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
+
     this.capacity = capacity;
     this.absoluteCapacity = absoluteCapacity;
     this.maximumCapacity = maximumCapacity;
@@ -162,9 +165,9 @@ public class ParentQueue implements CSQueue {
 
     this.acls = acls;
     
-    this.queueInfo.setCapacity(capacity);
-    this.queueInfo.setMaximumCapacity(maximumCapacity);
-    this.queueInfo.setQueueState(state);
+    this.queueInfo.setCapacity(this.capacity);
+    this.queueInfo.setMaximumCapacity(this.maximumCapacity);
+    this.queueInfo.setQueueState(this.state);
 
     StringBuilder aclsString = new StringBuilder();
     for (Map.Entry<QueueACL, AccessControlList> e : acls.entrySet()) {
@@ -484,6 +487,9 @@ public class ParentQueue implements CSQueue {
    * @param maximumCapacity new max capacity
    */
   synchronized void setMaxCapacity(float maximumCapacity) {
+    // Sanity check
+    CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
+    
     this.maximumCapacity = maximumCapacity;
     float parentAbsoluteCapacity = 
         (rootQueue) ? 100.0f : parent.getAbsoluteCapacity();

+ 4 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.java

@@ -61,6 +61,10 @@ public class RmView extends TwoColumnLayout {
     StringBuilder init = tableInit().
         append(", aoColumns:[{sType:'title-numeric'}, null, null, null, null,").
         append("null,{sType:'title-numeric', bSearchable:false}, null, null]");
+
+    // Sort by id upon page load
+    init.append(", aaSorting: [[0, 'asc']]");
+
     String rows = $("rowlimit");
     int rowLimit = rows.isEmpty() ? MAX_DISPLAY_ROWS : Integer.parseInt(rows);
     if (list.apps.size() < rowLimit) {

+ 46 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import junit.framework.Assert;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
@@ -35,7 +37,6 @@ public class TestQueueParsing {
 
     CapacityScheduler capacityScheduler = new CapacityScheduler();
     capacityScheduler.reinitialize(conf, null, null);
-    //capacityScheduler.g
   }
   
   private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
@@ -104,4 +105,48 @@ public class TestQueueParsing {
     CapacityScheduler capacityScheduler = new CapacityScheduler();
     capacityScheduler.reinitialize(conf, null, null);
   }
+  
+  public void testMaxCapacity() throws Exception {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+
+    conf.setQueues(CapacityScheduler.ROOT, new String[] {"a", "b", "c"});
+    conf.setCapacity(CapacityScheduler.ROOT, 100);
+
+    final String A = CapacityScheduler.ROOT + ".a";
+    conf.setCapacity(A, 50);
+    conf.setMaximumCapacity(A, 60);
+
+    final String B = CapacityScheduler.ROOT + ".b";
+    conf.setCapacity(B, 50);
+    conf.setMaximumCapacity(B, 45);  // Should throw an exception
+
+
+    boolean fail = false;
+    CapacityScheduler capacityScheduler;
+    try {
+      capacityScheduler = new CapacityScheduler();
+      capacityScheduler.reinitialize(conf, null, null);
+    } catch (IllegalArgumentException iae) {
+      fail = true;
+    }
+    Assert.assertTrue("Didn't throw IllegalArgumentException for wrong maxCap", 
+        fail);
+
+    conf.setMaximumCapacity(B, 60);
+    
+    // Now this should work
+    capacityScheduler = new CapacityScheduler();
+    capacityScheduler.reinitialize(conf, null, null);
+    
+    fail = false;
+    try {
+    LeafQueue a = (LeafQueue)capacityScheduler.getQueue(A);
+    a.setMaxCapacity(45);
+    } catch  (IllegalArgumentException iae) {
+      fail = true;
+    }
+    Assert.assertTrue("Didn't throw IllegalArgumentException for wrong " +
+    		"setMaxCap", fail);
+  }
+  
 }

+ 9 - 9
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/Federation.apt.vm

@@ -12,7 +12,7 @@
 ~~ limitations under the License. See accompanying LICENSE file.
 
   ---
-  Hadoop Map Reduce Next Generation-${project.version} - Cluster Setup
+  Hadoop Distributed File System-${project.version} - Federation
   ---
   ---
   ${maven.build.timestamp}
@@ -57,12 +57,12 @@ HDFS Federation
     * Storage - is provided by datanodes by storing blocks on the local file 
       system and allows read/write access.
 
-  The current HDFS architecture allows only a single namespace for the 
+  The prior HDFS architecture allows only a single namespace for the 
   entire cluster. A single Namenode manages this namespace. HDFS 
-  Federation addresses limitation of current architecture by adding 
+  Federation addresses limitation of the prior architecture by adding 
   support multiple Namenodes/namespaces to HDFS file system.
     
-* {HDFS Federation}
+* {Multiple Namenodes/Namespaces}
 
   In order to scale the name service horizontally, federation uses multiple 
   independent Namenodes/namespaces. The Namenodes are federated, that is, the 
@@ -103,9 +103,9 @@ HDFS Federation
     of small files benefit from scaling the namespace by adding more 
     Namenodes to the cluster
 
-  * Performance - File system operation throughput is currently limited 
-    by a single Namenode. Adding more Namenodes to the cluster scales the 
-    file system read/write operations throughput.
+  * Performance - File system operation throughput is limited by a single
+    Namenode in the prior architecture. Adding more Namenodes to the cluster
+    scales the file system read/write operations throughput.
 
   * Isolation - A single Namenode offers no isolation in multi user 
     environment. An experimental application can overload the Namenode 
@@ -265,7 +265,7 @@ HDFS Federation
 > $HADOOP_PREFIX_HOME/bin/start-dfs.sh
 ----
 
-  To start the cluster run the following command:
+  To stop the cluster run the following command:
 
 ----
 > $HADOOP_PREFIX_HOME/bin/stop-dfs.sh
@@ -300,7 +300,7 @@ HDFS Federation
 ** Decommissioning
 
   Decommissioning is similar to prior releases. The nodes that need to be 
-  decommissioning are added to the exclude file at all the Namenode. Each 
+  decomissioned are added to the exclude file at all the Namenode. Each 
   Namenode decommissions its Block Pool. When all the Namenodes finish 
   decommissioning a datanode, the datanode is considered to be decommissioned.
 

+ 1626 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WebHDFS.apt.vm

@@ -0,0 +1,1626 @@
+~~ Licensed 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. See accompanying LICENSE file.
+
+  ---
+  Hadoop Distributed File System-${project.version} - WebHDFS REST API
+  ---
+  ---
+  ${maven.build.timestamp}
+
+WebHDFS REST API
+
+  \[ {{{./index.html}Go Back}} \]
+
+%{toc|section=1|fromDepth=0}
+
+* {Document Conventions}
+
+*----------------------+-------------------------------------------------------------------------------+
+| <<<Monospaced>>>     | Used for commands, HTTP request and responses and code blocks.                |
+*----------------------+-------------------------------------------------------------------------------+
+| <<<\<Monospaced\>>>> | User entered values.                                                          |
+*----------------------+-------------------------------------------------------------------------------+
+| <<<[Monospaced]>>>   | Optional values.  When the value is not specified, the default value is used. |
+*----------------------+-------------------------------------------------------------------------------+
+| <Italics>            | Important phrases and words.                                                  |
+*----------------------+-------------------------------------------------------------------------------+
+
+
+* {Introduction}
+
+  The HTTP REST API supports the complete
+  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}} interface for HDFS.
+  The operations and the corresponding FileSystem methods are shown in the next section.
+  The Section {{HTTP Query Parameter Dictionary}} specifies the parameter details
+  such as the defaults and the valid values.
+
+** {Operations}
+
+  * HTTP GET
+
+    * {{{Open and Read a File}<<<OPEN>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.open)
+
+    * {{{Status of a File/Directory}<<<GETFILESTATUS>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getFileStatus)
+
+    * {{<<<LISTSTATUS>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.listStatus)
+
+    * {{{Get Content Summary of a Directory}<<<GETCONTENTSUMMARY>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getContentSummary)
+
+    * {{{Get File Checksum}<<<GETFILECHECKSUM>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getFileChecksum)
+
+    * {{{Get Home Directory}<<<GETHOMEDIRECTORY>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getHomeDirectory)
+
+    * {{{Get Delegation Token}<<<GETDELEGATIONTOKEN>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getDelegationToken)
+
+  * HTTP PUT
+
+    * {{{Create and Write to a File}<<<CREATE>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.create)
+
+    * {{{Make a Directory}<<<MKDIRS>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.mkdirs)
+
+    * {{{Rename a File/Directory}<<<RENAME>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.rename)
+
+    * {{{Set Replication Factor}<<<SETREPLICATION>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.setReplication)
+
+    * {{{Set Owner}<<<SETOWNER>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.setOwner)
+
+    * {{{Set Permission}<<<SETPERMISSION>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.setPermission)
+
+    * {{{Set Access or Modification Time}<<<SETTIMES>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.setTimes)
+
+    * {{{Renew Delegation Token}<<<RENEWDELEGATIONTOKEN>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.renewDelegationToken)
+
+    * {{{Cancel Delegation Token}<<<CANCELDELEGATIONTOKEN>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.cancelDelegationToken)
+
+  * HTTP POST
+
+    * {{{Append to a File}<<<APPEND>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.append)
+
+  * HTTP DELETE
+
+    * {{{Delete a File/Directory}<<<DELETE>>>}}
+        (see  {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.delete)
+
+** {FileSystem URIs vs HTTP URLs}
+
+  The FileSystem scheme of WebHDFS is "<<<webhdfs://>>>".
+  A WebHDFS FileSystem URI has the following format.
+
++---------------------------------
+  webhdfs://<HOST>:<HTTP_PORT>/<PATH>
++---------------------------------
+
+  The above WebHDFS URI corresponds to the below HDFS URI.
+
++---------------------------------
+  hdfs://<HOST>:<RPC_PORT>/<PATH>
++---------------------------------
+
+  In the REST API, the prefix "<<</webhdfs/v1>>>" is inserted in the path and a query is appended at the end.
+  Therefore, the corresponding HTTP URL has the following format.
+
++---------------------------------
+  http://<HOST>:<HTTP_PORT>/webhdfs/v1/<PATH>?op=...
++---------------------------------
+
+* {Authentication}
+
+  When security is <off>, the authenticated user is the username specified in the <<<user.name>>> query parameter.
+  If the <<<user.name>>> parameter is not set,
+  the server may either set the authenticated user to a default web user, if there is any, or return an error response.
+
+
+  When security is <on>, authentication is performed by either Hadoop delegation token or Kerberos SPNEGO.
+  If a token is set in the <<<delegation>>> query parameter, the authenticated user is the user encoded in the token.
+  If the <<<delegation>>> parameter is not set, the user is authenticated by Kerberos SPNEGO.
+
+
+  Below are examples using the <<<curl>>> command tool.
+
+  [[1]] Authentication when security is off:
+
++---------------------------------
+curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?[user.name=<USER>&]op=..."
++---------------------------------
+ 
+  [[1]] Authentication using Kerberos SPNEGO when security is on:
+
++---------------------------------
+curl -i --negotiate -u : "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=..."
++---------------------------------
+ 
+  [[1]] Authentication using Hadoop delegation token when security is on:
+
++---------------------------------
+curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?delegation=<TOKEN>&op=..."
++---------------------------------
+
+* {Proxy Users}
+
+  When the proxy user feature is enabled, a proxy user <P> may submit a request on behalf of another user <U>.
+  The username of <U> must be specified in the <<<doas>>> query parameter unless a delegation token is presented in authentication.
+  In such case, the information of both users <P> and <U> must be encoded in the delegation token.
+
+  [[1]] A proxy request when security is off:
+
++---------------------------------
+curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?[user.name=<USER>&]doas=<USER>&op=..."
++---------------------------------
+
+  [[1]] A proxy request using Kerberos SPNEGO when security is on:
+
++---------------------------------
+curl -i --negotiate -u : "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?doas=<USER>&op=..."
++---------------------------------
+
+  [[1]] A proxy request using Hadoop delegation token when security is on:
+
++---------------------------------
+curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?delegation=<TOKEN>&op=..."
++---------------------------------
+
+
+* {File and Directory Operations}
+
+** {Create and Write to a File}
+
+  * Step 1: Submit a HTTP PUT request without automatically following redirects and without sending the file data.
+
++---------------------------------
+curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=CREATE
+                    [&overwrite=<true|false>][&blocksize=<LONG>][&replication=<SHORT>]
+                    [&permission=<OCTAL>][&buffersize=<INT>]"
++---------------------------------
+
+  The request is redirected to a datanode where the file data is to be written:
+
++---------------------------------
+HTTP/1.1 307 TEMPORARY_REDIRECT
+Location: http://<DATANODE>:<PORT>/webhdfs/v1/<PATH>?op=CREATE...
+Content-Length: 0
++---------------------------------
+
+  * Step 2: Submit another HTTP PUT request using the URL in the <<<Location>>> header with the file data to be written.
+
++---------------------------------
+curl -i -X PUT -T <LOCAL_FILE> "http://<DATANODE>:<PORT>/webhdfs/v1/<PATH>?op=CREATE..."
++---------------------------------
+
+  The client receives a <<<201 Created>>> response with zero content length
+  and the WebHDFS URI of the file in the <<<Location>>> header:
+
++---------------------------------
+HTTP/1.1 201 Created
+Location: webhdfs://<HOST>:<PORT>/<PATH>
+Content-Length: 0
++---------------------------------
+
+  []
+
+  <<Note>> that the reason of having two-step create/append is
+  for preventing clients to send out data before the redirect.
+  This issue is addressed by the "<<<Expect: 100-continue>>>" header in HTTP/1.1;
+  see {{{http://www.w3.org/Protocols/rfc2616/rfc2616-sec8.html#sec8.2.3}RFC 2616, Section 8.2.3}}.
+  Unfortunately, there are software library bugs (e.g. Jetty 6 HTTP server and Java 6 HTTP client),
+  which do not correctly implement "<<<Expect: 100-continue>>>".
+  The two-step create/append is a temporary workaround for the software library bugs.
+
+  See also:
+  {{{Overwrite}<<<overwrite>>>}},
+  {{{Block Size}<<<blocksize>>>}},
+  {{{Replication}<<<replication>>>}},
+  {{{Permission}<<<permission>>>}},
+  {{{Buffer Size}<<<buffersize>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.create
+
+
+** {Append to a File}
+
+  * Step 1: Submit a HTTP POST request without automatically following redirects and without sending the file data.
+
++---------------------------------
+curl -i -X POST "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=APPEND[&buffersize=<INT>]"
++---------------------------------
+
+  The request is redirected to a datanode where the file data is to be appended:
+
++---------------------------------
+HTTP/1.1 307 TEMPORARY_REDIRECT
+Location: http://<DATANODE>:<PORT>/webhdfs/v1/<PATH>?op=APPEND...
+Content-Length: 0
++---------------------------------
+
+  * Step 2: Submit another HTTP POST request using the URL in the <<<Location>>> header with the file data to be appended.
+
++---------------------------------
+curl -i -X POST -T <LOCAL_FILE> "http://<DATANODE>:<PORT>/webhdfs/v1/<PATH>?op=APPEND..."
++---------------------------------
+
+  The client receives a response with zero content length:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Length: 0
++---------------------------------
+
+  []
+
+  See the note in the previous section for the description of why this operation requires two steps.
+
+  See also:
+  {{{Buffer Size}<<<buffersize>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.append
+
+
+** {Open and Read a File}
+
+  * Submit a HTTP GET request with automatically following redirects.
+
++---------------------------------
+curl -i -L "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=OPEN
+                    [&offset=<LONG>][&length=<LONG>][&buffersize=<INT>]"
++---------------------------------
+
+  The request is redirected to a datanode where the file data can be read:
+
++---------------------------------
+HTTP/1.1 307 TEMPORARY_REDIRECT
+Location: http://<DATANODE>:<PORT>/webhdfs/v1/<PATH>?op=OPEN...
+Content-Length: 0
++---------------------------------
+
+  The client follows the redirect to the datanode and receives the file data:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/octet-stream
+Content-Length: 22
+
+Hello, webhdfs user!
++---------------------------------
+
+  []
+
+  See also:
+  {{{Offset}<<<offset>>>}},
+  {{{Length}<<<length>>>}},
+  {{{Buffer Size}<<<buffersize>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.open
+
+
+** {Make a Directory}
+
+  * Submit a HTTP PUT request.
+
++---------------------------------
+curl -i -X PUT "http://<HOST>:<PORT>/<PATH>?op=MKDIRS[&permission=<OCTAL>]"
++---------------------------------
+
+  The client receives a response with a {{{Boolean JSON Schema}<<<boolean>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"boolean": true}
++---------------------------------
+
+  []
+
+  See also:
+  {{{Permission}<<<permission>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.mkdirs
+
+
+** {Rename a File/Directory}
+
+  * Submit a HTTP PUT request.
+
++---------------------------------
+curl -i -X PUT "<HOST>:<PORT>/webhdfs/v1/<PATH>?op=RENAME&destination=<PATH>"
++---------------------------------
+
+  The client receives a response with a {{{Boolean JSON Schema}<<<boolean>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"boolean": true}
++---------------------------------
+
+  []
+
+  See also:
+  {{{Destination}<<<destination>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.rename
+
+
+** {Delete a File/Directory}
+
+  * Submit a HTTP DELETE request.
+
++---------------------------------
+curl -i -X DELETE "http://<host>:<port>/webhdfs/v1/<path>?op=DELETE
+                              [&recursive=<true|false>]"
++---------------------------------
+
+  The client receives a response with a {{{Boolean JSON Schema}<<<boolean>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"boolean": true}
++---------------------------------
+
+  []
+
+  See also:
+  {{{Recursive}<<<recursive>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.delete
+
+
+** {Status of a File/Directory}
+
+  * Submit a HTTP GET request.
+
++---------------------------------
+curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETFILESTATUS"
++---------------------------------
+
+  The client receives a response with a {{{FileStatus JSON Schema}<<<FileStatus>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "FileStatus":
+  {
+    "accessTime"      : 0,
+    "blockSize"       : 0,
+    "group"           : "supergroup",
+    "length"          : 0,             //in bytes, zero for directories
+    "modificationTime": 1320173277227,
+    "owner"           : "webuser",
+    "pathSuffix"      : "",
+    "permission"      : "777",
+    "replication"     : 0,
+    "type"            : "DIRECTORY"    //enum {FILE, DIRECTORY, SYMLINK}
+  }
+}
++---------------------------------
+
+  []
+
+  See also:
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getFileStatus
+
+
+** {List a Directory}
+
+  * Submit a HTTP GET request.
+
++---------------------------------
+curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTSTATUS"
++---------------------------------
+
+  The client receives a response with a {{{FileStatuses JSON Schema}<<<FileStatuses>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Content-Length: 427
+
+{
+  "FileStatuses":
+  {
+    "FileStatus":
+    [
+      {
+        "accessTime"      : 1320171722771,
+        "blockSize"       : 33554432,
+        "group"           : "supergroup",
+        "length"          : 24930,
+        "modificationTime": 1320171722771,
+        "owner"           : "webuser",
+        "pathSuffix"      : "a.patch",
+        "permission"      : "644",
+        "replication"     : 1,
+        "type"            : "FILE"
+      },
+      {
+        "accessTime"      : 0,
+        "blockSize"       : 0,
+        "group"           : "supergroup",
+        "length"          : 0,
+        "modificationTime": 1320895981256,
+        "owner"           : "szetszwo",
+        "pathSuffix"      : "bar",
+        "permission"      : "711",
+        "replication"     : 0,
+        "type"            : "DIRECTORY"
+      },
+      ...
+    ]
+  }
+}
++---------------------------------
+
+  []
+
+  See also:
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.listStatus
+
+
+* {Other File System Operations}
+
+** {Get Content Summary of a Directory}
+
+  * Submit a HTTP GET request.
+
++---------------------------------
+curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETCONTENTSUMMARY"
++---------------------------------
+
+  The client receives a response with a {{{ContentSummary JSON Schema}<<<ContentSummary>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "ContentSummary":
+  {
+    "directoryCount": 2,
+    "fileCount"     : 1,
+    "length"        : 24930,
+    "quota"         : -1,
+    "spaceConsumed" : 24930,
+    "spaceQuota"    : -1
+  }
+}
++---------------------------------
+
+  []
+
+  See also:
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getContentSummary
+
+
+** {Get File Checksum}
+
+  * Submit a HTTP GET request.
+
++---------------------------------
+curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETFILECHECKSUM"
++---------------------------------
+
+  The request is redirected to a datanode:
+
++---------------------------------
+HTTP/1.1 307 TEMPORARY_REDIRECT
+Location: http://<DATANODE>:<PORT>/webhdfs/v1/<PATH>?op=GETFILECHECKSUM...
+Content-Length: 0
++---------------------------------
+
+  The client follows the redirect to the datanode and receives a {{{FileChecksum JSON Schema}<<<FileChecksum>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "FileChecksum":
+  {
+    "algorithm": "MD5-of-1MD5-of-512CRC32",
+    "bytes"    : "eadb10de24aa315748930df6e185c0d ...",
+    "length"   : 28
+  }
+}
++---------------------------------
+
+  []
+
+  See also:
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getFileChecksum
+
+
+** {Get Home Directory}
+
+  * Submit a HTTP GET request.
+
++---------------------------------
+curl -i "http://<HOST>:<PORT>/webhdfs/v1/?op=GETHOMEDIRECTORY"
++---------------------------------
+
+  The client receives a response with a {{{Path JSON Schema}<<<Path>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"Path": "/user/szetszwo"}
++---------------------------------
+
+  []
+
+  See also:
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getHomeDirectory
+
+
+** {Set Permission}
+
+  * Submit a HTTP PUT request.
+
++---------------------------------
+curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=SETPERMISSION
+                              [&permission=<OCTAL>]"
++---------------------------------
+
+  The client receives a response with zero content length:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Length: 0
++---------------------------------
+
+  []
+
+  See also:
+  {{{Permission}<<<permission>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.setPermission
+
+
+** {Set Owner}
+
+  * Submit a HTTP PUT request.
+
++---------------------------------
+curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=SETOWNER
+                              [&owner=<USER>][&group=<GROUP>]"
++---------------------------------
+
+  The client receives a response with zero content length:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Length: 0
++---------------------------------
+
+  []
+
+  See also:
+  {{{Owner}<<<owner>>>}},
+  {{{Group}<<<group>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.setOwner
+
+
+** {Set Replication Factor}
+
+  * Submit a HTTP PUT request.
+
++---------------------------------
+curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=SETREPLICATION
+                              [&replication=<SHORT>]"
++---------------------------------
+
+  The client receives a response with a {{{Boolean JSON Schema}<<<boolean>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"boolean": true}
++---------------------------------
+
+  []
+
+  See also:
+  {{{Replication}<<<replication>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.setReplication
+
+
+** {Set Access or Modification Time}
+
+  * Submit a HTTP PUT request.
+
++---------------------------------
+curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=SETTIMES
+                              [&modificationtime=<TIME>][&accesstime=<TIME>]"
++---------------------------------
+
+  The client receives a response with zero content length:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Length: 0
++---------------------------------
+
+  []
+
+  See also:
+  {{{Modification Time}<<<modificationtime>>>}},
+  {{{Access Time}<<<accesstime>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.setTimes
+
+
+* {Delegation Token Operations}
+
+** {Get Delegation Token}
+
+  * Submit a HTTP GET request.
+
++---------------------------------
+curl -i "http://<HOST>:<PORT>/webhdfs/v1/?op=GETDELEGATIONTOKEN&renewer=<USER>"
++---------------------------------
+
+  The client receives a response with a {{{Token JSON Schema}<<<Token>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "Token":
+  {
+    "urlString": "JQAIaG9y..."
+  }
+}
++---------------------------------
+
+  []
+
+  See also:
+  {{{Renewer}<<<renewer>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.getDelegationToken
+
+
+** {Renew Delegation Token}
+
+  * Submit a HTTP PUT request.
+
++---------------------------------
+curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=<TOKEN>"
++---------------------------------
+
+  The client receives a response with a {{{Long JSON Schema}<<<long>>> JSON object}}:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{"long": 1320962673997}           //the new expiration time
++---------------------------------
+
+  []
+
+  See also:
+  {{{Token}<<<token>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.renewDelegationToken
+
+
+** {Cancel Delegation Token}
+
+  * Submit a HTTP PUT request.
+
++---------------------------------
+curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/?op=CANCELDELEGATIONTOKEN&token=<TOKEN>"
++---------------------------------
+
+  The client receives a response with zero content length:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Length: 0
++---------------------------------
+
+  []
+
+  See also:
+  {{{Token}<<<token>>>}},
+   {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.cancelDelegationToken
+
+
+* {Error Responses}
+
+  When an operation fails, the server may throw an exception.
+  The JSON schema of error responses is defined in {{<<<RemoteException>>> JSON schema}}.
+  The table below shows the mapping from exceptions to HTTP response codes.
+
+** {HTTP Response Codes}
+
+*-------------------------------------+---------------------------------+
+|| Exceptions                         || HTTP Response Codes            |
+*-------------------------------------+---------------------------------+
+| <<<IllegalArgumentException     >>> | <<<400 Bad Request          >>> |
+*-------------------------------------+---------------------------------+
+| <<<UnsupportedOperationException>>> | <<<400 Bad Request          >>> |
+*-------------------------------------+---------------------------------+
+| <<<SecurityException            >>> | <<<401 Unauthorized         >>> |
+*-------------------------------------+---------------------------------+
+| <<<IOException                  >>> | <<<403 Forbidden            >>> |
+*-------------------------------------+---------------------------------+
+| <<<FileNotFoundException        >>> | <<<404 Not Found            >>> |
+*-------------------------------------+---------------------------------+
+| <<<RumtimeException             >>> | <<<500 Internal Server Error>>> |
+*-------------------------------------+---------------------------------+
+
+  Below are examples of exception responses.
+
+*** {Illegal Argument Exception}
+
++---------------------------------
+HTTP/1.1 400 Bad Request
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "RemoteException":
+  {
+    "exception"    : "IllegalArgumentException",
+    "javaClassName": "java.lang.IllegalArgumentException",
+    "message"      : "Invalid value for webhdfs parameter \"permission\": ..."
+  }
+}
++---------------------------------
+
+
+*** {Security Exception}
+
++---------------------------------
+HTTP/1.1 401 Unauthorized
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "RemoteException":
+  {
+    "exception"    : "SecurityException",
+    "javaClassName": "java.lang.SecurityException",
+    "message"      : "Failed to obtain user group information: ..."
+  }
+}
++---------------------------------
+
+
+*** {Access Control Exception}
+
++---------------------------------
+HTTP/1.1 403 Forbidden
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "RemoteException":
+  {
+    "exception"    : "AccessControlException",
+    "javaClassName": "org.apache.hadoop.security.AccessControlException",
+    "message"      : "Permission denied: ..."
+  }
+}
++---------------------------------
+
+
+*** {File Not Found Exception}
+
++---------------------------------
+HTTP/1.1 404 Not Found
+Content-Type: application/json
+Transfer-Encoding: chunked
+
+{
+  "RemoteException":
+  {
+    "exception"    : "FileNotFoundException",
+    "javaClassName": "java.io.FileNotFoundException",
+    "message"      : "File does not exist: /foo/a.patch"
+  }
+}
++---------------------------------
+
+
+* {JSON Schemas}
+
+  All operations, except for {{{Open and Read a File}<<<OPEN>>>}},
+  either return a zero-length response or a JSON response. 
+  For {{{Open and Read a File}<<<OPEN>>>}}, the response is an octet-stream.
+  The JSON schemas are shown below.
+  See {{{http://tools.ietf.org/id/draft-zyp-json-schema-03.html}draft-zyp-json-schema-03}}
+  for the syntax definitions of the JSON schemas.
+
+
+** {Boolean JSON Schema}
+
++---------------------------------
+{
+  "name"      : "boolean",
+  "properties":
+  {
+    "boolean":
+    {
+      "description": "A boolean value",
+      "type"       : "boolean",
+      "required"   : true
+    }
+  }
+}
++---------------------------------
+
+  See also:
+  {{{Make a Directory}<<<MKDIRS>>>}},
+  {{{Rename a File/Directory}<<<RENAME>>>}},
+  {{{Delete a File/Directory}<<<DELETE>>>}},
+  {{{Set Replication Factor}<<<SETREPLICATION>>>}}
+
+
+** {ContentSummary JSON Schema}
+
++---------------------------------
+{
+  "name"      : "ContentSummary",
+  "properties":
+  {
+    "ContentSummary":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "directoryCount":
+        {
+          "description": "The number of directories.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "fileCount":
+        {
+          "description": "The number of files.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "length":
+        {
+          "description": "The number of bytes used by the content.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "quota":
+        {
+          "description": "The namespace quota of this directory.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "spaceConsumed":
+        {
+          "description": "The disk space consumed by the content.",
+          "type"       : "integer",
+          "required"   : true
+        },
+        "spaceQuota":
+        {
+          "description": "The disk space quota.",
+          "type"       : "integer",
+          "required"   : true
+        }
+      }
+    }
+  }
+}
++---------------------------------
+
+  See also:
+  {{{Get Content Summary of a Directory}<<<GETCONTENTSUMMARY>>>}}
+
+
+** {FileChecksum JSON Schema}
+
++---------------------------------
+{
+  "name"      : "FileChecksum",
+  "properties":
+  {
+    "FileChecksum":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "algorithm":
+        {
+          "description": "The name of the checksum algorithm.",
+          "type"       : "string",
+          "required"   : true
+        },
+        "bytes":
+        {
+          "description": "The byte sequence of the checksum in hexadecimal.",
+          "type"       : "string",
+          "required"   : true
+        },
+        "length":
+        {
+          "description": "The length of the bytes (not the length of the string).",
+          "type"       : "integer",
+          "required"   : true
+        }
+      }
+    }
+  }
+}
++---------------------------------
+
+  See also:
+  {{{Get File Checksum}<<<GETFILECHECKSUM>>>}}
+
+
+** {FileStatus JSON Schema}
+
++---------------------------------
+{
+  "name"      : "FileStatus",
+  "properties":
+  {
+    "FileStatus": fileStatusProperties      //See FileStatus Properties
+  }
+}
++---------------------------------
+
+  See also:
+  {{{FileStatus Properties}<<<FileStatus>>> Properties}},
+  {{{Status of a File/Directory}<<<GETFILESTATUS>>>}},
+  {{{../../api/org/apache/hadoop/fs/FileStatus}FileStatus}}
+
+
+*** {FileStatus Properties}
+
+  JavaScript syntax is used to define <<<fileStatusProperties>>>
+  so that it can be referred in both <<<FileStatus>>> and <<<FileStatuses>>> JSON schemas.
+
++---------------------------------
+var fileStatusProperties =
+{
+  "type"      : "object",
+  "properties":
+  {
+    "accessTime":
+    {
+      "description": "The access time.",
+      "type"       : "integer",
+      "required"   : true
+    },
+    "blockSize":
+    {
+      "description": "The block size of a file.",
+      "type"       : "integer",
+      "required"   : true
+    },
+    "group":
+    {
+      "description": "The group owner.",
+      "type"       : "string",
+      "required"   : true
+    },
+    "length":
+    {
+      "description": "The number of bytes in a file.",
+      "type"       : "integer",
+      "required"   : true
+    },
+    "modificationTime":
+    {
+      "description": "The modification time.",
+      "type"       : "integer",
+      "required"   : true
+    },
+    "owner":
+    {
+      "description": "The user who is the owner.",
+      "type"       : "string",
+      "required"   : true
+    },
+    "pathSuffix":
+    {
+      "description": "The path suffix.",
+      "type"       : "string",
+      "required"   : true
+    },
+    "permission":
+    {
+      "description": "The permission represented as a octal string.",
+      "type"       : "string",
+      "required"   : true
+    },
+    "replication":
+    {
+      "description": "The number of replication of a file.",
+      "type"       : "integer",
+      "required"   : true
+    },
+   "symlink":                                         //an optional property
+    {
+      "description": "The link target of a symlink.",
+      "type"       : "string"
+    },
+   "type":
+    {
+      "description": "The type of the path object.",
+      "enum"       : ["FILE", "DIRECTORY", "SYMLINK"],
+      "required"   : true
+    }
+  }
+};
++---------------------------------
+
+
+** {FileStatuses JSON Schema}
+
+  A <<<FileStatuses>>> JSON object represents an array of <<<FileStatus>>> JSON objects.
+
++---------------------------------
+{
+  "name"      : "FileStatuses",
+  "properties":
+  {
+    "FileStatuses":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "FileStatus":
+        {
+          "description": "An array of FileStatus",
+          "type"       : "array",
+          "items"      : fileStatusProperties      //See FileStatus Properties
+        }
+      }
+    }
+  }
+}
++---------------------------------
+
+  See also:
+  {{{FileStatus Properties}<<<FileStatus>>> Properties}},
+  {{{List a Directory}<<<LISTSTATUS>>>}},
+  {{{../../api/org/apache/hadoop/fs/FileStatus}FileStatus}}
+
+
+** {Long JSON Schema}
+
++---------------------------------
+{
+  "name"      : "long",
+  "properties":
+  {
+    "long":
+    {
+      "description": "A long integer value",
+      "type"       : "integer",
+      "required"   : true
+    }
+  }
+}
++---------------------------------
+
+  See also:
+  {{{Renew Delegation Token}<<<RENEWDELEGATIONTOKEN>>>}},
+
+
+** {Path JSON Schema}
+
++---------------------------------
+{
+  "name"      : "Path",
+  "properties":
+  {
+    "Path":
+    {
+      "description": "The string representation a Path.",
+      "type"       : "string",
+      "required"   : true
+    }
+  }
+}
++---------------------------------
+
+  See also:
+  {{{Get Home Directory}<<<GETHOMEDIRECTORY>>>}},
+  {{{../../api/org/apache/hadoop/fs/Path}Path}}
+
+
+** {RemoteException JSON Schema}
+
++---------------------------------
+{
+  "name"      : "RemoteException",
+  "properties":
+  {
+    "RemoteException":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "exception":
+        {
+          "description": "Name of the exception",
+          "type"       : "string",
+          "required"   : true
+        },
+        "message":
+        {
+          "description": "Exception message",
+          "type"       : "string",
+          "required"   : true
+        },
+        "javaClassName":                                     //an optional property
+        {
+          "description": "Java class name of the exception",
+          "type"       : "string",
+        }
+      }
+    }
+  }
+}
++---------------------------------
+
+  See also:
+  {{Error Responses}}
+
+
+** {Token JSON Schema}
+
++---------------------------------
+{
+  "name"      : "Token",
+  "properties":
+  {
+    "Token":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "urlString":
+        {
+          "description": "A delegation token encoded as a URL safe string.",
+          "type"       : "string",
+          "required"   : true
+        }
+      }
+    }
+  }
+}
++---------------------------------
+
+  See also:
+  {{{Get Delegation Token}<<<GETDELEGATIONTOKEN>>>}},
+  the note in {{Delegation}}.
+
+
+* {HTTP Query Parameter Dictionary}
+
+** {Access Time}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<accesstime>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The access time of a file/directory. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | long |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | -1 (means keeping it unchanged) |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | -1 or a timestamp |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any integer. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Set Access or Modification Time}<<<SETTIMES>>>}}
+
+
+** {Block Size}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<blocksize>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The block size of a file. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | long |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | Specified in the configuration. |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | \> 0 |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any integer. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Create and Write to a File}<<<CREATE>>>}}
+
+
+** {Buffer Size}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<buffersize>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The size of the buffer used in transferring data. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | int |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | Specified in the configuration. |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | \> 0 |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any integer. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Create and Write to a File}<<<CREATE>>>}},
+  {{{Append to a File}<<<APPEND>>>}},
+  {{{Open and Read a File}<<<OPEN>>>}}
+
+
+** {Delegation}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<delegation>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The delegation token used for authentication. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | String |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | \<empty\> |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | An encoded token. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | See the note below. |
+*----------------+-------------------------------------------------------------------+
+
+  <<Note>> that delegation tokens are encoded as a URL safe string;
+  see <<<encodeToUrlString()>>>
+  and <<<decodeFromUrlString(String)>>>
+  in <<<org.apache.hadoop.security.token.Token>>> for the details of the encoding.
+
+
+  See also:
+  {{Authentication}}
+
+
+** {Destination}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<destination>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The destination path used in {{{Rename a File/Directory}<<<RENAME>>>}}. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | Path |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | \<empty\> (an invalid path) |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | An absolute FileSystem path without scheme and authority. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any path. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Rename a File/Directory}<<<RENAME>>>}}
+
+
+** {Do As}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<doas>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | Allowing a proxy user to do as another user. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | String |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | null |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | Any valid username. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any string. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{Proxy Users}}
+
+
+** {Group}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<group>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The name of a group. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | String |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | \<empty\> (means keeping it unchanged) |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | Any valid group name. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any string. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Set Owner}<<<SETOWNER>>>}}
+
+
+** {Length}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<length>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The number of bytes to be processed. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | long |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | null (means the entire file) |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | \>= 0 or null |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any integer. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Open and Read a File}<<<OPEN>>>}}
+
+
+** {Modification Time}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<modificationtime>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The modification time of a file/directory. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | long |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | -1 (means keeping it unchanged) |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | -1 or a timestamp |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any integer. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Set Access or Modification Time}<<<SETTIMES>>>}}
+
+
+** {Offset}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<offset>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The starting byte position. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | long |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | 0 |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | \>= 0 |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any integer. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Open and Read a File}<<<OPEN>>>}}
+
+
+** {Op}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<op>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The name of the operation to be executed. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | enum |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | null (an invalid value) |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | Any valid operation name. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any string. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{Operations}}
+
+
+** {Overwrite}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<overwrite>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | If a file already exists, should it be overwritten? |
+*----------------+-------------------------------------------------------------------+
+|| Type          | boolean |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | false |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | true | false |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | true | false |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Create and Write to a File}<<<CREATE>>>}}
+
+
+** {Owner}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<owner>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The username who is the owner of a file/directory. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | String |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | \<empty\> (means keeping it unchanged) |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | Any valid username. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any string. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Set Owner}<<<SETOWNER>>>}}
+
+
+** {Permission}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<permission>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The permission of a file/directory. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | Octal |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | 755 |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | 0 - 1777 |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any radix-8 integer (leading zeros may be omitted.) |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Create and Write to a File}<<<CREATE>>>}},
+  {{{Make a Directory}<<<MKDIRS>>>}},
+  {{{Set Permission}<<<SETPERMISSION>>>}}
+
+
+** {Recursive}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<recursive>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | Should the operation act on the content in the subdirectories? |
+*----------------+-------------------------------------------------------------------+
+|| Type          | boolean |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | false |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | true | false |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | true | false |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Rename a File/Directory}<<<RENAME>>>}}
+
+
+** {Renewer}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<renewer>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The username of the renewer of a delegation token. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | String |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | \<empty\> (means the current user) |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | Any valid username. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any string. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Get Delegation Token}<<<GETDELEGATIONTOKEN>>>}}
+
+
+** {Replication}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<replication>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The number of replications of a file. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | short |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | Specified in the configuration. |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | \> 0 |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any integer. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Create and Write to a File}<<<CREATE>>>}},
+  {{{Set Replication Factor}<<<SETREPLICATION>>>}}
+
+
+** {Token}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<token>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The delegation token used for the operation. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | String |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | \<empty\> |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | An encoded token. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | See the note in {{Delegation}}. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{{Renew Delegation Token}<<<RENEWDELEGATIONTOKEN>>>}},
+  {{{Cancel Delegation Token}<<<CANCELDELEGATIONTOKEN>>>}}
+
+
+** {Username}
+
+*----------------+-------------------------------------------------------------------+
+|| Name          | <<<user.name>>> |
+*----------------+-------------------------------------------------------------------+
+|| Description   | The authenticated user; see {{Authentication}}. |
+*----------------+-------------------------------------------------------------------+
+|| Type          | String |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | null |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values  | Any valid username. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax        | Any string. |
+*----------------+-------------------------------------------------------------------+
+
+  See also:
+  {{Authentication}}
+

+ 4 - 0
hadoop-mapreduce-project/src/contrib/gridmix/ivy.xml

@@ -64,6 +64,10 @@
                rev="${yarn.version}" conf="common->default"/>
    <dependency org="org.apache.hadoop" name="hadoop-yarn-common"
                rev="${yarn.version}" conf="common->default"/>
+   <dependency org="org.apache.hadoop" name="hadoop-mapreduce-client-jobclient"
+               rev="${yarn.version}" conf="test->default">
+     <artifact name="hadoop-mapreduce-client-jobclient" type="tests" ext="jar" m:classifier="tests"/>
+   </dependency>
    <dependency org="commons-logging"
       name="commons-logging"
       rev="${commons-logging.version}"

+ 0 - 52
hadoop-mapreduce-project/src/contrib/streaming/build.xml

@@ -1,52 +0,0 @@
-<?xml version="1.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.
--->
-
-<!-- 
-Before you can run these subtargets directly, you need 
-to call at top-level: ant deploy-contrib compile-core-test
--->
-<project name="streaming" default="jar">
-
-  <import file="../build-contrib.xml"/>
-
-  <!-- Override jar target to specify main class -->
-  <target name="jar" depends="compile">
-    <jar
-      jarfile="${build.dir}/hadoop-${version}-${name}.jar"
-      basedir="${build.classes}"      
-    >
-  	<manifest>
-	    <attribute name="Main-Class" value="org.apache.hadoop.streaming.HadoopStreaming"/>
-	</manifest>
-    </jar>
-  </target>
-
-  <!-- Run all unit tests. superdottest -->
-  <target name="test">
-   <antcall target="hadoopbuildcontrib.test"> 
-   </antcall>
-  </target>  
- 
-  <!--Run all system tests.-->
-  <target name="test-system">
-    <antcall target="hadoopbuildcontrib.test-system">
-    </antcall>
-  </target>
-
-</project>

+ 0 - 98
hadoop-mapreduce-project/src/contrib/streaming/ivy.xml

@@ -1,98 +0,0 @@
-<?xml version="1.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.
--->
-
-<ivy-module version="1.0" xmlns:m="http://ant.apache.org/ivy/maven">
-  <info organisation="org.apache.hadoop" module="${ant.project.name}">
-    <license name="Apache 2.0"/>
-    <ivyauthor name="Apache Hadoop Team" url="http://hadoop.apache.org"/>
-    <description>
-        Apache Hadoop
-    </description>
-  </info>
-  <configurations defaultconfmapping="default">
-    <!--these match the Maven configurations-->
-    <conf name="default" extends="master,runtime"/>
-    <conf name="master" description="contains the artifact but no dependencies"/>
-    <conf name="runtime" description="runtime but not the artifact" />
-
-    <conf name="common" visibility="private" 
-      extends="runtime"
-      description="artifacts needed to compile/test the application"/>
-    <conf name="test" visibility="private" extends="runtime"/>
-  </configurations>
-
-  <publications>
-    <!--get the artifact from our module name-->
-    <artifact conf="master"/>
-  </publications>
-  <dependencies>
-   <dependency org="org.apache.hadoop" name="hadoop-annotations" rev="${hadoop-common.version}" conf="common->default"/>
-    <dependency org="org.apache.hadoop" name="hadoop-common" 
-                rev="${hadoop-common.version}" conf="common->default"/>
-    <dependency org="org.apache.hadoop" name="hadoop-common" 
-                rev="${hadoop-common.version}" conf="test->default">
-      <artifact name="hadoop-common" type="tests" ext="jar" m:classifier="tests"/>
-    </dependency>
-    <dependency org="org.apache.hadoop" name="hadoop-hdfs" 
-                rev="${hadoop-hdfs.version}" conf="common->default"/>
-    <dependency org="org.apache.hadoop" name="hadoop-hdfs"
-                rev="${hadoop-hdfs.version}" conf="test->default">
-      <artifact name="hadoop-hdfs" type="tests" ext="jar" m:classifier="tests"/>
-    </dependency>
-   <dependency org="org.apache.hadoop" name="hadoop-mapreduce-client-core" 
-               rev="${yarn.version}" conf="common->default"/>
-   <dependency org="org.apache.hadoop" name="hadoop-yarn-common"
-               rev="${yarn.version}" conf="common->default"/>
-
-    <dependency org="commons-cli" name="commons-cli" 
-                rev="${commons-cli.version}" conf="common->default"/>
-    <dependency org="commons-logging" name="commons-logging" 
-                rev="${commons-logging.version}" conf="common->default"/>
-    <dependency org="junit" name="junit" 
-                rev="${junit.version}" conf="common->default"/>
-    <dependency org="org.mortbay.jetty" name="jetty-util"
-                rev="${jetty-util.version}" conf="common->master"/>
-    <dependency org="org.mortbay.jetty" name="jetty" 
-                rev="${jetty.version}" conf="common->master"/>
-    <dependency org="org.mortbay.jetty" name="jsp-api-2.1" 
-                rev="${jetty.version}" conf="common->master"/>
-    <dependency org="org.mortbay.jetty" name="jsp-2.1" 
-                rev="${jetty.version}" conf="common->master"/>
-    <dependency org="org.mortbay.jetty" name="servlet-api-2.5" 
-                rev="${servlet-api-2.5.version}" conf="common->master"/>
-    <dependency org="commons-httpclient" name="commons-httpclient" 
-                rev="${commons-httpclient.version}" conf="common->default"/>
-    <dependency org="log4j" name="log4j" 
-                rev="${log4j.version}" conf="common->master"/>
-    <dependency org="org.apache.avro" name="avro" 
-                rev="${avro.version}" conf="common->default">
-      <exclude module="ant"/>
-      <exclude module="jetty"/>
-      <exclude module="slf4j-simple"/>
-    </dependency>
-    <dependency org="org.slf4j" name="slf4j-api" 
-                rev="${slf4j-api.version}" conf="common->master"/>
-
-   <!-- Exclusions for transitive dependencies pulled in by log4j -->
-   <exclude org="com.sun.jdmk"/>
-   <exclude org="com.sun.jmx"/>
-   <exclude org="javax.jms"/> 
-   <exclude org="javax.mail"/>
-
-  </dependencies>
-</ivy-module>

+ 0 - 17
hadoop-mapreduce-project/src/contrib/streaming/ivy/libraries.properties

@@ -1,17 +0,0 @@
-#   Licensed 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 properties file lists the versions of the various artifacts used by streaming.
-#It drives ivy and the generation of a maven POM
-
-#Please list the dependencies name with version if they are different from the ones 
-#listed in the global libraries.properties file (in alphabetical order)

+ 5 - 1
hadoop-mapreduce-project/src/docs/changes/changes2html.pl

@@ -242,7 +242,11 @@ for my $rel (@releases) {
 
       $item =~ s:\s*(\([^)"]+?\))\s*$:<br />$1:;       # Separate attribution
       $item =~ s:\n{2,}:\n<p/>\n:g;                    # Keep paragraph breaks
-      $item =~ s{(?:${jira_url_prefix})?(HADOOP-\d+)}  # Link to JIRA
+      $item =~ s{(?:${jira_url_prefix})?(HADOOP-\d+)}  # Link to JIRA Common
+                {<a href="${jira_url_prefix}$1">$1</a>}g;
+      $item =~ s{(?:${jira_url_prefix})?(HDFS-\d+)}    # Link to JIRA Hdfs
+                {<a href="${jira_url_prefix}$1">$1</a>}g;
+      $item =~ s{(?:${jira_url_prefix})?(MAPREDUCE-\d+)}  # Link to JIRA MR
                 {<a href="${jira_url_prefix}$1">$1</a>}g;
       print "      <li>$item</li>\n";
     }

+ 1 - 3
hadoop-mapreduce-project/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml

@@ -2060,9 +2060,7 @@
           <p>Hadoop comes configured with a single mandatory queue, called 
           'default'. Queue names are defined in the 
           <code>mapred.queue.names</code> property of the Hadoop site
-          configuration. Some job schedulers, such as the 
-          <a href="capacity_scheduler.html">Capacity Scheduler</a>, 
-          support multiple queues.</p>
+          configuration.</p>
           
           <p>A job defines the queue it needs to be submitted to through the
           <code>mapreduce.job.queuename</code> property.

+ 1 - 0
hadoop-mapreduce-project/src/webapps/job/analysejobhistory.jsp

@@ -35,6 +35,7 @@
 %>
 <%!	private static final long serialVersionUID = 1L;
 %>
+<!DOCTYPE html>
 <html><body>
 <%
   String logFile = request.getParameter("logFile");

+ 1 - 0
hadoop-mapreduce-project/src/webapps/job/job_authorization_error.jsp

@@ -28,6 +28,7 @@
 <%!	private static final long serialVersionUID = 1L;
 %>
 
+<!DOCTYPE html>
 <html>
 <head>
 <title>Error: User cannot access this Job</title>

+ 1 - 0
hadoop-mapreduce-project/src/webapps/job/jobblacklistedtrackers.jsp

@@ -74,6 +74,7 @@
     }
 %>
 
+<!DOCTYPE html>
 <html>
 <title>Hadoop <%=jobId%>'s black-listed tasktrackers</title>
 <body>

+ 1 - 1
hadoop-mapreduce-project/src/webapps/job/jobconf.jsp

@@ -39,7 +39,7 @@
     return;
   }
 %>
-  
+<!DOCTYPE html>  
 <html>
 
 <title>Job Configuration: JobId - <%= jobId %></title>

+ 1 - 0
hadoop-mapreduce-project/src/webapps/job/jobconf_history.jsp

@@ -49,6 +49,7 @@
 
 %>
   
+<!DOCTYPE html>
 <html>
 
 <title>Job Configuration: JobId - <%= jobId %></title>

+ 1 - 0
hadoop-mapreduce-project/src/webapps/job/jobdetails.jsp

@@ -267,6 +267,7 @@
 %>
 
 <%@page import="org.apache.hadoop.mapred.TaskGraphServlet"%>
+<!DOCTYPE html>
 <html>
 <head>
   <% 

+ 1 - 0
hadoop-mapreduce-project/src/webapps/job/jobdetailshistory.jsp

@@ -60,6 +60,7 @@
       reasonforFailure = job.getErrorInfo();
 %>
 
+<!DOCTYPE html>
 <html>
 <head>
 <title>Hadoop Job <%=jobid%> on History Viewer</title>

Algunos archivos no se mostraron porque demasiados archivos cambiaron en este cambio