浏览代码

Merge trunk into HDFS-1623 branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1204794 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 年之前
父节点
当前提交
6a0671977b
共有 100 个文件被更改,包括 28007 次插入2314 次删除
  1. 10 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
  4. 26 24
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  5. 4 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
  6. 12 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
  7. 97 16
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextSymlinkBaseTest.java
  8. 8 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestPath.java
  9. 54 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  10. 380 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  11. 107 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  12. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  13. 19 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  14. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  15. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
  16. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  17. 97 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java
  18. 28 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  19. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  20. 25 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
  21. 11652 1124
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/HdfsProtos.java
  22. 2516 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/InterDatanodeProtocolProtos.java
  23. 2234 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/JournalProtocolProtos.java
  24. 8949 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/NamenodeProtocolProtos.java
  25. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolServerSideTranslatorR23.java
  26. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolTranslatorR23.java
  27. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeWireProtocol.java
  28. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  29. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  30. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  31. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
  32. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
  33. 782 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  34. 12 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  35. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  36. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  37. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
  38. 139 710
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  39. 1 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  40. 14 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
  41. 18 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  42. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
  43. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
  44. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
  45. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
  46. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
  47. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
  48. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
  49. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  50. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  51. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  52. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
  53. 0 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java
  54. 21 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  55. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
  56. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  57. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  58. 11 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  59. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  60. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  61. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  62. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  63. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RenewDelegationTokenServlet.java
  64. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
  65. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockCommandWritable.java
  66. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
  67. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
  68. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoader.java
  69. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitor.java
  70. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitorFactory.java
  71. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java
  72. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DirectBufferPool.java
  73. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java
  74. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  75. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/proto/InterDatanodeProtocol.proto
  76. 83 0
      hadoop-hdfs-project/hadoop-hdfs/src/proto/JournalProtocol.proto
  77. 225 0
      hadoop-hdfs-project/hadoop-hdfs/src/proto/NamenodeProtocol.proto
  78. 162 6
      hadoop-hdfs-project/hadoop-hdfs/src/proto/hdfs.proto
  79. 0 41
      hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj
  80. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java
  81. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
  82. 0 97
      hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java
  83. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestHelperDFS.java
  84. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestHDFSCLI.java
  85. 12 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSymlink.java
  86. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
  87. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
  88. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
  89. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  90. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java
  91. 56 53
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  92. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java
  93. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java
  94. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
  95. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
  96. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java
  97. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java
  98. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  99. 4 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
  100. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java

+ 10 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -60,6 +60,8 @@ Trunk (unreleased changes)
     HADOOP-7688. Add servlet handler check in HttpServer.start().
     (Uma Maheswara Rao G via szetszwo)
 
+    HADOOP-7590. Mavenize streaming and MR examples. (tucu)
+
   BUGS
 
     HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
@@ -93,6 +95,9 @@ Trunk (unreleased changes)
     HADOOP-7770. ViewFS getFileChecksum throws FileNotFoundException for files in 
     /tmp and /user. (Ravi Prakash via jitendra)
 
+    HADOOP-7833. Fix findbugs warnings in protobuf generated code.
+    (John Lee via suresh)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -123,6 +128,9 @@ Release 0.23.1 - Unreleased
    HADOOP-7787. Make source tarball use conventional name.
    (Bruno Mahé via tomwhite)
 
+   HADOOP-6614. RunJar should provide more diags when it can't create
+   a temp file. (Jonathan Hsieh via eli)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES
@@ -1356,6 +1364,8 @@ Release 0.22.0 - Unreleased
     HADOOP-7457. Remove out-of-date Chinese language documentation.
     (Jakob Homan via eli)
 
+    HADOOP-7783. Add more symlink tests that cover intermediate links. (eli)
+
 Release 0.21.1 - Unreleased
 
   IMPROVEMENTS

+ 1 - 1
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -272,6 +272,6 @@
     </Match>
     <Match>
       <!-- protobuf generated code -->
-      <Class name="org.apache.hadoop.ipc.protobuf.HadoopRpcProtos"/>
+      <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.HadoopRpcProtos.*"/>
     </Match>
  </FindBugsFilter>

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java

@@ -385,7 +385,7 @@ public abstract class AbstractFileSystem {
     checkPath(p);
     String s = p.toUri().getPath();
     if (!isValidName(s)) {
-      throw new InvalidPathException("Path part " + s + " from URI" + p
+      throw new InvalidPathException("Path part " + s + " from URI " + p
           + " is not a valid filename.");
     }
     return s;

+ 26 - 24
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java

@@ -1092,29 +1092,28 @@ public final class FileContext {
    * Return a fully qualified version of the given symlink target if it
    * has no scheme and authority. Partially and fully qualified paths 
    * are returned unmodified.
-   * @param linkFS The AbstractFileSystem of link 
-   * @param link   The path of the symlink
-   * @param target The symlink's target
+   * @param pathFS The AbstractFileSystem of the path
+   * @param pathWithLink Path that contains the symlink
+   * @param target The symlink's absolute target
    * @return Fully qualified version of the target.
    */
-  private Path qualifySymlinkTarget(final AbstractFileSystem linkFS, 
-      Path link, Path target) {
-    /* NB: makeQualified uses link's scheme/authority, if specified, 
-     * and the scheme/authority of linkFS, if not. If link does have
-     * a scheme and authority they should match those of linkFS since
-     * resolve updates the path and file system of a path that contains
-     * links each time a link is encountered.
+  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.
      */
-    final String linkScheme = link.toUri().getScheme();
-    final String linkAuth   = link.toUri().getAuthority();
-    if (linkScheme != null && linkAuth != null) {
-      assert linkScheme.equals(linkFS.getUri().getScheme());
-      assert linkAuth.equals(linkFS.getUri().getAuthority());
+    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());
     }
-    final boolean justPath = (target.toUri().getScheme() == null &&
-                              target.toUri().getAuthority() == null);
-    return justPath ? target.makeQualified(linkFS.getUri(), link.getParent()) 
-                    : target;
+    return (scheme == null && auth == null)
+      ? target.makeQualified(pathFS.getUri(), pathWithLink.getParent())
+      : target;
   }
   
   /**
@@ -1148,16 +1147,19 @@ public final class FileContext {
   }
   
   /**
-   * Returns the un-interpreted target of the given symbolic link.
-   * Transparently resolves all links up to the final path component.
-   * @param f
+   * Returns the target of the given symbolic link as it was specified
+   * when the link was created.  Links in the path leading up to the
+   * final path component are resolved transparently.
+   *
+   * @param f the path to return the target of
    * @return The un-interpreted target of the symbolic link.
    * 
    * @throws AccessControlException If access is denied
    * @throws FileNotFoundException If path <code>f</code> does not exist
    * @throws UnsupportedFileSystemException If file system for <code>f</code> is
    *           not supported
-   * @throws IOException If an I/O error occurred
+   * @throws IOException If the given path does not refer to a symlink
+   *           or an I/O error occurred
    */
   public Path getLinkTarget(final Path f) throws AccessControlException,
       FileNotFoundException, UnsupportedFileSystemException, IOException {
@@ -1277,7 +1279,7 @@ public final class FileContext {
    * getFsStatus, getFileStatus, exists, and listStatus.
    * 
    * Symlink targets are stored as given to createSymlink, assuming the 
-   * underlying file system is capable of storign a fully qualified URI. 
+   * underlying file system is capable of storing a fully qualified URI.
    * Dangling symlinks are permitted. FileContext supports four types of 
    * symlink targets, and resolves them as follows
    * <pre>

+ 4 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java

@@ -68,13 +68,14 @@ public class Path implements Comparable {
     // Add a slash to parent's path so resolution is compatible with URI's
     URI parentUri = parent.uri;
     String parentPath = parentUri.getPath();
-    if (!(parentPath.equals("/") || parentPath.equals("")))
+    if (!(parentPath.equals("/") || parentPath.equals(""))) {
       try {
         parentUri = new URI(parentUri.getScheme(), parentUri.getAuthority(),
                       parentUri.getPath()+"/", null, parentUri.getFragment());
       } catch (URISyntaxException e) {
         throw new IllegalArgumentException(e);
       }
+    }
     URI resolved = parentUri.resolve(child.uri);
     initialize(resolved.getScheme(), resolved.getAuthority(),
                resolved.getPath(), resolved.getFragment());
@@ -213,7 +214,8 @@ public class Path implements Comparable {
    * There is some ambiguity here. An absolute path is a slash
    * relative name without a scheme or an authority.
    * So either this method was incorrectly named or its
-   * implementation is incorrect.
+   * implementation is incorrect. This method returns true
+   * even if there is a scheme and authority.
    */
   public boolean isAbsolute() {
      return isUriPathAbsolute();
@@ -307,19 +309,16 @@ public class Path implements Comparable {
     return depth;
   }
 
-  
   /**
    *  Returns a qualified path object.
    *  
    *  Deprecated - use {@link #makeQualified(URI, Path)}
    */
- 
   @Deprecated
   public Path makeQualified(FileSystem fs) {
     return makeQualified(fs.getUri(), fs.getWorkingDirectory());
   }
   
-  
   /** Returns a qualified path object. */
   @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
   public Path makeQualified(URI defaultUri, Path workingDir ) {

+ 12 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java

@@ -149,7 +149,18 @@ public class RunJar {
     File tmpDir = new File(new Configuration().get("hadoop.tmp.dir"));
     ensureDirectory(tmpDir);
 
-    final File workDir = File.createTempFile("hadoop-unjar", "", tmpDir);
+    final File workDir;
+    try { 
+      workDir = File.createTempFile("hadoop-unjar", "", tmpDir);
+    } catch (IOException ioe) {
+      // If user has insufficient perms to write to tmpDir, default  
+      // "Permission denied" message doesn't specify a filename. 
+      System.err.println("Error creating temp dir in hadoop.tmp.dir "
+                         + tmpDir + " due to " + ioe.getMessage());
+      System.exit(-1);
+      return;
+    }
+
     if (!workDir.delete()) {
       System.err.println("Delete failed for " + workDir);
       System.exit(-1);

+ 97 - 16
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextSymlinkBaseTest.java

@@ -28,8 +28,9 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
-import static org.junit.Assert.*;
 
+import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 import org.junit.Test;
 import org.junit.Before;
 import org.junit.After;
@@ -238,6 +239,31 @@ public abstract class FileContextSymlinkBaseTest {
     assertFalse(isDir(fc, linkToFile));
     assertEquals(file.toUri().getPath(), 
                  fc.getLinkTarget(linkToFile).toString());
+    // The local file system does not fully resolve the link
+    // when obtaining the file status
+    if (!"file".equals(getScheme())) {
+      assertEquals(fc.getFileStatus(file), fc.getFileStatus(linkToFile));
+      assertEquals(fc.makeQualified(file),
+                   fc.getFileStatus(linkToFile).getPath());
+      assertEquals(fc.makeQualified(linkToFile),
+                   fc.getFileLinkStatus(linkToFile).getPath());
+    }
+  }
+
+  @Test
+  /** Stat a relative link to a file */
+  public void testStatRelLinkToFile() throws IOException {
+    assumeTrue(!"file".equals(getScheme()));
+    Path baseDir    = new Path(testBaseDir1());
+    Path file       = new Path(testBaseDir1(), "file");
+    Path linkToFile = new Path(testBaseDir1(), "linkToFile");
+    createAndWriteFile(file);
+    fc.createSymlink(new Path("file"), linkToFile, false);
+    assertEquals(fc.getFileStatus(file), fc.getFileStatus(linkToFile));
+    assertEquals(fc.makeQualified(file),
+                 fc.getFileStatus(linkToFile).getPath());
+    assertEquals(fc.makeQualified(linkToFile),
+                 fc.getFileLinkStatus(linkToFile).getPath());
   }
 
   @Test
@@ -474,18 +500,15 @@ public abstract class FileContextSymlinkBaseTest {
    * creating using a partially qualified path is file system specific.
    */
   public void testCreateLinkUsingPartQualPath1() throws IOException {
+    // Partially qualified paths are covered for local file systems
+    // in the previous test.
+    assumeTrue(!"file".equals(getScheme()));
     Path schemeAuth   = new Path(testURI().toString());
     Path fileWoHost   = new Path(getScheme()+"://"+testBaseDir1()+"/file");
     Path link         = new Path(testBaseDir1()+"/linkToFile");
     Path linkQual     = new Path(schemeAuth, testBaseDir1()+"/linkToFile");
-    
-    // Partially qualified paths are covered for local file systems
-    // in the previous test.
-    if ("file".equals(getScheme())) {
-      return;
-    }
     FileContext localFc = FileContext.getLocalFSFileContext();
-    
+
     fc.createSymlink(fileWoHost, link, false);
     // Partially qualified path is stored
     assertEquals(fileWoHost, fc.getLinkTarget(linkQual));    
@@ -748,7 +771,7 @@ public abstract class FileContextSymlinkBaseTest {
   }
 
   @Test
-  /** Test create symlink to ../foo */
+  /** Test create symlink to ../file */
   public void testCreateLinkToDotDotPrefix() throws IOException {
     Path file = new Path(testBaseDir1(), "file");
     Path dir  = new Path(testBaseDir1(), "test");
@@ -1205,24 +1228,30 @@ public abstract class FileContextSymlinkBaseTest {
   }
   
   @Test
-  /** Operate on a file using a path with an intermediate symlink */  
-  public void testAccessFileViaSymlink() throws IOException {
+  /**
+   * Create, write, read, append, rename, get the block locations,
+   * checksums, and delete a file using a path with a symlink as an
+   * intermediate path component where the link target was specified
+   * using an absolute path. Rename is covered in more depth below.
+   */
+  public void testAccessFileViaInterSymlinkAbsTarget() throws IOException {
     Path baseDir        = new Path(testBaseDir1());
+    Path file           = new Path(testBaseDir1(), "file");
     Path fileNew        = new Path(baseDir, "fileNew");
     Path linkToDir      = new Path(testBaseDir2(), "linkToDir");
     Path fileViaLink    = new Path(linkToDir, "file");
     Path fileNewViaLink = new Path(linkToDir, "fileNew");
     fc.createSymlink(baseDir, linkToDir, false);
-    // Create, write, read, append, rename, get block locations and 
-    // checksums, and delete a file using a path that contains a 
-    // symlink as an intermediate path component. Rename is covered 
-    // in more depth below.
     createAndWriteFile(fileViaLink);
     assertTrue(exists(fc, fileViaLink));
     assertTrue(isFile(fc, fileViaLink));
     assertFalse(isDir(fc, fileViaLink));
     assertFalse(fc.getFileLinkStatus(fileViaLink).isSymlink());
     assertFalse(isDir(fc, fileViaLink));
+    assertEquals(fc.getFileStatus(file),
+                 fc.getFileLinkStatus(file));
+    assertEquals(fc.getFileStatus(fileViaLink),
+                 fc.getFileLinkStatus(fileViaLink));
     readFile(fileViaLink);
     appendToFile(fileViaLink);
     fc.rename(fileViaLink, fileNewViaLink);
@@ -1237,6 +1266,58 @@ public abstract class FileContextSymlinkBaseTest {
     assertFalse(exists(fc, fileNewViaLink));
   }
 
+  @Test
+  /**
+   * Operate on a file using a path with an intermediate symlink where
+   * the link target was specified as a fully qualified path.
+   */
+  public void testAccessFileViaInterSymlinkQualTarget() throws IOException {
+    Path baseDir        = new Path(testBaseDir1());
+    Path file           = new Path(testBaseDir1(), "file");
+    Path fileNew        = new Path(baseDir, "fileNew");
+    Path linkToDir      = new Path(testBaseDir2(), "linkToDir");
+    Path fileViaLink    = new Path(linkToDir, "file");
+    Path fileNewViaLink = new Path(linkToDir, "fileNew");
+    fc.createSymlink(fc.makeQualified(baseDir), linkToDir, false);
+    createAndWriteFile(fileViaLink);
+    assertEquals(fc.getFileStatus(file),
+                 fc.getFileLinkStatus(file));
+    assertEquals(fc.getFileStatus(fileViaLink),
+                 fc.getFileLinkStatus(fileViaLink));
+    readFile(fileViaLink);
+  }
+
+  @Test
+  /**
+   * Operate on a file using a path with an intermediate symlink where
+   * the link target was specified as a relative path.
+   */
+  public void testAccessFileViaInterSymlinkRelTarget() throws IOException {
+    assumeTrue(!"file".equals(getScheme()));
+    Path baseDir     = new Path(testBaseDir1());
+    Path dir         = new Path(testBaseDir1(), "dir");
+    Path file        = new Path(dir, "file");
+    Path linkToDir   = new Path(testBaseDir1(), "linkToDir");
+    Path fileViaLink = new Path(linkToDir, "file");
+
+    fc.mkdir(dir, FileContext.DEFAULT_PERM, false);
+    fc.createSymlink(new Path("dir"), linkToDir, false);
+    createAndWriteFile(fileViaLink);
+    // Note that getFileStatus returns fully qualified paths even
+    // when called on an absolute path.
+    assertEquals(fc.makeQualified(file),
+                 fc.getFileStatus(file).getPath());
+    // In each case getFileLinkStatus returns the same FileStatus
+    // as getFileStatus since we're not calling it on a link and
+    // FileStatus objects are compared by Path.
+    assertEquals(fc.getFileStatus(file),
+                 fc.getFileLinkStatus(file));
+    assertEquals(fc.getFileStatus(fileViaLink),
+                 fc.getFileLinkStatus(fileViaLink));
+    assertEquals(fc.getFileStatus(fileViaLink),
+                 fc.getFileLinkStatus(file));
+  }
+
   @Test
   /** Test create, list, and delete a directory through a symlink */
   public void testAccessDirViaSymlink() throws IOException {
@@ -1272,4 +1353,4 @@ public abstract class FileContextSymlinkBaseTest {
       assertEquals(2, fc.getFileStatus(file).getModificationTime());
     }
   }
-}
+}

+ 8 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestPath.java

@@ -95,6 +95,7 @@ public class TestPath extends TestCase {
     assertEquals(new Path("/foo"), new Path("/foo/bar").getParent());
     assertEquals(new Path("foo"), new Path("foo/bar").getParent());
     assertEquals(new Path("/"), new Path("/foo").getParent());
+    assertEquals(null, new Path("/").getParent());
     if (Path.WINDOWS) {
       assertEquals(new Path("c:/"), new Path("c:/foo").getParent());
     }
@@ -159,6 +160,13 @@ public class TestPath extends TestCase {
     assertEquals(new Path("foo/bar/baz","../../..").toString(), "");
     assertEquals(new Path("foo/bar/baz","../../../../..").toString(), "../..");
   }
+
+  /** Test Path objects created from other Path objects */
+  public void testChildParentResolution() throws URISyntaxException, IOException {
+    Path parent = new Path("foo1://bar1/baz1");
+    Path child  = new Path("foo2://bar2/baz2");
+    assertEquals(child, new Path(parent, child));
+  }
   
   public void testScheme() throws java.io.IOException {
     assertEquals("foo:/bar", new Path("foo:/","/bar").toString()); 

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

@@ -5,6 +5,12 @@ Trunk (unreleased changes)
     HDFS-395.  DFS Scalability: Incremental block reports. (Tomasz Nykiel
                via hairong)
 
+    HDFS-2517. Add protobuf service for JounralProtocol. (suresh)
+
+    HDFS-2518. Add protobuf service for NamenodeProtocol. (suresh)
+
+    HDFS-2520. Add protobuf service for InterDatanodeProtocol. (suresh)
+
   IMPROVEMENTS
 
     HADOOP-7524 Change RPC to allow multiple protocols including multuple 
@@ -52,6 +58,13 @@ Trunk (unreleased changes)
 
     HDFS-2334. Add Closeable to JournalManager. (Ivan Kelly via jitendra)
 
+    HDFS-2564. Cleanup unnecessary exceptions thrown and unnecessary casts.
+    (Hari Mankude via eli)
+
+    HDFS-2572. Remove unnecessary double-check in DN#getHostName. (harsh)
+
+    HDFS-2410. Further cleanup of hardcoded configuration keys and values.
+    (suresh)
 
   OPTIMIZATIONS
     HDFS-2477. Optimize computing the diff between a block report and the
@@ -100,6 +113,9 @@ Trunk (unreleased changes)
     HDFS-2526. (Client)NamenodeProtocolTranslatorR23 do not need to keep a
                reference to rpcProxyWithoutRetry (atm)
 
+    HDFS-2532. TestDfsOverAvroRpc timing out in trunk (Uma Maheswara Rao G
+               via todd)
+
 Release 0.23.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -117,6 +133,24 @@ Release 0.23.1 - UNRELEASED
     HDFS-2562. Refactor DN configuration variables out of DataNode class
     (todd)
 
+    HDFS-2563. Some cleanup in BPOfferService. (todd)
+
+    HDFS-208. name node should warn if only one dir is listed in dfs.name.dir.
+    (Uma Maheswara Rao G via eli)
+
+    HDFS-2568. Use a set to manage child sockets in XceiverServer.
+    (harsh via eli)
+
+    HDFS-2454. Move maxXceiverCount check to before starting the
+    thread in dataXceiver. (harsh via eli)
+
+    HDFS-2570. Add descriptions for dfs.*.https.address in hdfs-default.xml.
+    (eli)
+
+    HDFS-2536. Remove unused imports. (harsh via eli)
+
+    HDFS-2566. Move BPOfferService to be a non-inner class. (todd)
+
   OPTIMIZATIONS
 
     HDFS-2130. Switch default checksum to CRC32C. (todd)
@@ -127,8 +161,23 @@ Release 0.23.1 - UNRELEASED
     HDFS-2129. Simplify BlockReader to not inherit from FSInputChecker.
     (todd)
 
+    HDFS-2246. Enable reading a block directly from local file system
+    for a client on the same node as the block file.  (Andrew Purtell,
+    Suresh Srinivas and Jitendra Nath Pandey via szetszwo)
+
   BUG FIXES
 
+    HDFS-2541. For a sufficiently large value of blocks, the DN Scanner 
+    may request a random number with a negative seed value. (harsh via eli)
+
+    HDFS-2502. hdfs-default.xml should include dfs.name.dir.restore.
+    (harsh via eli)
+
+    HDFS-2567. When 0 DNs are available, show a proper error when
+    trying to browse DFS via web UI. (harsh via eli)
+
+    HDFS-2575. DFSTestUtil may create empty files (todd)
+    
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES
@@ -1902,6 +1951,11 @@ Release 0.22.0 - Unreleased
     HDFS-2002. Incorrect computation of needed blocks in getTurnOffTip().
     (Plamen Jeliazkov via shv)
 
+    HDFS-2514. Link resolution bug for intermediate symlinks with
+    relative targets. (eli)
+
+    HDFS-2573. TestFiDataXceiverServer is failing, not testing OOME (cos)
+
 Release 0.21.1 - Unreleased
 
     HDFS-1466. TestFcHdfsSymlink relies on /tmp/test not existing. (eli)

+ 380 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -0,0 +1,380 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DataChecksum;
+
+/**
+ * BlockReaderLocal enables local short circuited reads. If the DFS client is on
+ * the same machine as the datanode, then the client can read files directly
+ * from the local file system rather than going through the datanode for better
+ * performance. <br>
+ * {@link BlockReaderLocal} works as follows:
+ * <ul>
+ * <li>The client performing short circuit reads must be configured at the
+ * datanode.</li>
+ * <li>The client gets the path to the file where block is stored using
+ * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
+ * RPC call</li>
+ * <li>Client uses kerberos authentication to connect to the datanode over RPC,
+ * if security is enabled.</li>
+ * </ul>
+ */
+class BlockReaderLocal extends RemoteBlockReader2 {
+  public static final Log LOG = LogFactory.getLog(DFSClient.class);
+
+  //Stores the cache and proxy for a local datanode.
+  private static class LocalDatanodeInfo {
+    private ClientDatanodeProtocol proxy = null;
+    private final Map<ExtendedBlock, BlockLocalPathInfo> cache;
+
+    LocalDatanodeInfo() {
+      final int cacheSize = 10000;
+      final float hashTableLoadFactor = 0.75f;
+      int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
+      cache = Collections
+          .synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
+              hashTableCapacity, hashTableLoadFactor, true) {
+            private static final long serialVersionUID = 1;
+
+            @Override
+            protected boolean removeEldestEntry(
+                Map.Entry<ExtendedBlock, BlockLocalPathInfo> eldest) {
+              return size() > cacheSize;
+            }
+          });
+    }
+
+    private synchronized ClientDatanodeProtocol getDatanodeProxy(
+        DatanodeInfo node, Configuration conf, int socketTimeout)
+        throws IOException {
+      if (proxy == null) {
+        proxy = DFSUtil.createClientDatanodeProtocolProxy(node, conf,
+            socketTimeout);
+      }
+      return proxy;
+    }
+    
+    private synchronized void resetDatanodeProxy() {
+      if (null != proxy) {
+        RPC.stopProxy(proxy);
+        proxy = null;
+      }
+    }
+
+    private BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) {
+      return cache.get(b);
+    }
+
+    private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
+      cache.put(b, info);
+    }
+
+    private void removeBlockLocalPathInfo(ExtendedBlock b) {
+      cache.remove(b);
+    }
+  }
+  
+  // Multiple datanodes could be running on the local machine. Store proxies in
+  // a map keyed by the ipc port of the datanode.
+  private static Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
+
+  private final FileInputStream dataIn; // reader for the data file
+
+  private FileInputStream checksumIn;   // reader for the checksum file
+
+  private int offsetFromChunkBoundary;
+  
+  ByteBuffer dataBuff = null;
+  ByteBuffer checksumBuff = null;
+  
+  /**
+   * The only way this object can be instantiated.
+   */
+  static BlockReaderLocal newBlockReader(Configuration conf, String file,
+      ExtendedBlock blk, Token<BlockTokenIdentifier> token, DatanodeInfo node,
+      int socketTimeout, long startOffset, long length) throws IOException {
+
+    LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
+        .getIpcPort());
+    // check the cache first
+    BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
+    if (pathinfo == null) {
+      pathinfo = getBlockPathInfo(blk, node, conf, socketTimeout, token);
+    }
+
+    // check to see if the file exists. It may so happen that the
+    // HDFS file has been deleted and this block-lookup is occurring
+    // on behalf of a new HDFS file. This time, the block file could
+    // be residing in a different portion of the fs.data.dir directory.
+    // In this case, we remove this entry from the cache. The next
+    // call to this method will re-populate the cache.
+    FileInputStream dataIn = null;
+    FileInputStream checksumIn = null;
+    BlockReaderLocal localBlockReader = null;
+    boolean skipChecksumCheck = skipChecksumCheck(conf);
+    try {
+      // get a local file system
+      File blkfile = new File(pathinfo.getBlockPath());
+      dataIn = new FileInputStream(blkfile);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("New BlockReaderLocal for file " + blkfile + " of size "
+            + blkfile.length() + " startOffset " + startOffset + " length "
+            + length + " short circuit checksum " + skipChecksumCheck);
+      }
+
+      if (!skipChecksumCheck) {
+        // get the metadata file
+        File metafile = new File(pathinfo.getMetaPath());
+        checksumIn = new FileInputStream(metafile);
+
+        // read and handle the common header here. For now just a version
+        BlockMetadataHeader header = BlockMetadataHeader
+            .readHeader(new DataInputStream(checksumIn));
+        short version = header.getVersion();
+        if (version != FSDataset.METADATA_VERSION) {
+          LOG.warn("Wrong version (" + version + ") for metadata file for "
+              + blk + " ignoring ...");
+        }
+        DataChecksum checksum = header.getChecksum();
+        long firstChunkOffset = startOffset
+            - (startOffset % checksum.getBytesPerChecksum());
+        localBlockReader = new BlockReaderLocal(conf, file, blk, token,
+            startOffset, length, pathinfo, checksum, true, dataIn,
+            firstChunkOffset, checksumIn);
+      } else {
+        localBlockReader = new BlockReaderLocal(conf, file, blk, token,
+            startOffset, length, pathinfo, dataIn);
+      }
+    } catch (IOException e) {
+      // remove from cache
+      localDatanodeInfo.removeBlockLocalPathInfo(blk);
+      DFSClient.LOG.warn("BlockReaderLocal: Removing " + blk
+          + " from cache because local file " + pathinfo.getBlockPath()
+          + " could not be opened.");
+      throw e;
+    } finally {
+      if (localBlockReader == null) {
+        if (dataIn != null) {
+          dataIn.close();
+        }
+        if (checksumIn != null) {
+          checksumIn.close();
+        }
+      }
+    }
+    return localBlockReader;
+  }
+  
+  private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
+    LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
+    if (ldInfo == null) {
+      ldInfo = new LocalDatanodeInfo();
+      localDatanodeInfoMap.put(port, ldInfo);
+    }
+    return ldInfo;
+  }
+  
+  private static BlockLocalPathInfo getBlockPathInfo(ExtendedBlock blk,
+      DatanodeInfo node, Configuration conf, int timeout,
+      Token<BlockTokenIdentifier> token) throws IOException {
+    LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.ipcPort);
+    BlockLocalPathInfo pathinfo = null;
+    ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(node,
+        conf, timeout);
+    try {
+      // make RPC to local datanode to find local pathnames of blocks
+      pathinfo = proxy.getBlockLocalPathInfo(blk, token);
+      if (pathinfo != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Cached location of block " + blk + " as " + pathinfo);
+        }
+        localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
+      }
+    } catch (IOException e) {
+      localDatanodeInfo.resetDatanodeProxy(); // Reset proxy on error
+      throw e;
+    }
+    return pathinfo;
+  }
+  
+  private static boolean skipChecksumCheck(Configuration conf) {
+    return conf.getBoolean(
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
+  }
+  
+  private BlockReaderLocal(Configuration conf, String hdfsfile,
+      ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
+      long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn)
+      throws IOException {
+    this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
+        DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_NULL, 4), false,
+        dataIn, startOffset, null);
+  }
+
+  private BlockReaderLocal(Configuration conf, String hdfsfile,
+      ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
+      long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
+      boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
+      FileInputStream checksumIn) throws IOException {
+    super(hdfsfile, block.getBlockPoolId(), block.getBlockId(), dataIn
+        .getChannel(), checksum, verifyChecksum, startOffset, firstChunkOffset,
+        length, null);
+    this.dataIn = dataIn;
+    this.checksumIn = checksumIn;
+    this.offsetFromChunkBoundary = (int) (startOffset-firstChunkOffset);
+    dataBuff = bufferPool.getBuffer(bytesPerChecksum*64);
+    checksumBuff = bufferPool.getBuffer(checksumSize*64);
+    //Initially the buffers have nothing to read.
+    dataBuff.flip();
+    checksumBuff.flip();
+    long toSkip = firstChunkOffset;
+    while (toSkip > 0) {
+      long skipped = dataIn.skip(toSkip);
+      if (skipped == 0) {
+        throw new IOException("Couldn't initialize input stream");
+      }
+      toSkip -= skipped;
+    }
+    if (checksumIn != null) {
+      long checkSumOffset = (firstChunkOffset / bytesPerChecksum)
+          * checksumSize;
+      while (checkSumOffset > 0) {
+        long skipped = checksumIn.skip(checkSumOffset);
+        if (skipped == 0) {
+          throw new IOException("Couldn't initialize checksum input stream");
+        }
+        checkSumOffset -= skipped;
+      }
+    }
+  }
+
+  private int readIntoBuffer(FileInputStream stream, ByteBuffer buf)
+      throws IOException {
+    int bytesRead = stream.getChannel().read(buf);
+    if (bytesRead < 0) {
+      //EOF
+      return bytesRead;
+    }
+    while (buf.remaining() > 0) {
+      int n = stream.getChannel().read(buf);
+      if (n < 0) {
+        //EOF
+        return bytesRead;
+      }
+      bytesRead += n;
+    }
+    return bytesRead;
+  }
+  
+  @Override
+  public synchronized int read(byte[] buf, int off, int len) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.info("read off " + off + " len " + len);
+    }
+    if (!verifyChecksum) {
+      return dataIn.read(buf, off, len);
+    } else {
+      int dataRead = -1;
+      if (dataBuff.remaining() == 0) {
+        dataBuff.clear();
+        checksumBuff.clear();
+        dataRead = readIntoBuffer(dataIn, dataBuff);
+        readIntoBuffer(checksumIn, checksumBuff);
+        checksumBuff.flip();
+        dataBuff.flip();
+        if (verifyChecksum) {
+          checksum.verifyChunkedSums(dataBuff, checksumBuff, filename,
+              this.startOffset);
+        }
+      } else {
+        dataRead = dataBuff.remaining();
+      }
+      if (dataRead > 0) {
+        int nRead = Math.min(dataRead - offsetFromChunkBoundary, len);
+        if (offsetFromChunkBoundary > 0) {
+          dataBuff.position(offsetFromChunkBoundary);
+          // Its either end of file or dataRead is greater than the
+          // offsetFromChunkBoundary
+          offsetFromChunkBoundary = 0;
+        }
+        if (nRead > 0) {
+          dataBuff.get(buf, off, nRead);
+          return nRead;
+        } else {
+          return 0;
+        }
+      } else {
+        return -1;
+      }
+    }
+  }
+
+  @Override
+  public synchronized long skip(long n) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("skip " + n);
+    }
+    if (!verifyChecksum) {
+      return dataIn.skip(n);
+    } else {
+     return super.skip(n);
+    }
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    dataIn.close();
+    if (checksumIn != null) {
+      checksumIn.close();
+    }
+    if (dataBuff != null) {
+      bufferPool.returnBuffer(dataBuff);
+      dataBuff = null;
+    }
+    if (checksumBuff != null) {
+      bufferPool.returnBuffer(checksumBuff);
+      checksumBuff = null;
+    }
+    super.close();
+  }
+}

+ 107 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -24,13 +24,20 @@ import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.net.NetworkInterface;
 import java.net.Socket;
+import java.net.SocketException;
 import java.net.URI;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+
 import javax.net.SocketFactory;
 
 import org.apache.commons.logging.Log;
@@ -77,6 +84,8 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
@@ -234,6 +243,8 @@ public class DFSClient implements java.io.Closeable {
   private final Map<String, DFSOutputStream> filesBeingWritten
       = new HashMap<String, DFSOutputStream>();
 
+  private boolean shortCircuitLocalReads;
+  
   /**
    * Same as this(NameNode.getAddress(conf), conf);
    * @see #DFSClient(InetSocketAddress, Configuration)
@@ -317,6 +328,13 @@ public class DFSClient implements java.io.Closeable {
           "Expecting exactly one of nameNodeUri and rpcNamenode being null: "
           + "nameNodeUri=" + nameNodeUri + ", rpcNamenode=" + rpcNamenode);
     }
+    // read directly from the block file if configured.
+    this.shortCircuitLocalReads = conf.getBoolean(
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
+        DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Short circuit read is " + shortCircuitLocalReads);
+    }
   }
   
   private Class<?> getFailoverProxyProviderClass(String authority, Configuration conf)
@@ -539,6 +557,82 @@ public class DFSClient implements java.io.Closeable {
     }
   }
 
+  /**
+   * Get {@link BlockReader} for short circuited local reads.
+   */
+  static BlockReader getLocalBlockReader(Configuration conf,
+      String src, ExtendedBlock blk, Token<BlockTokenIdentifier> accessToken,
+      DatanodeInfo chosenNode, int socketTimeout, long offsetIntoBlock)
+      throws InvalidToken, IOException {
+    try {
+      return BlockReaderLocal.newBlockReader(conf, src, blk, accessToken,
+          chosenNode, socketTimeout, offsetIntoBlock, blk.getNumBytes()
+              - offsetIntoBlock);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(InvalidToken.class,
+          AccessControlException.class);
+    }
+  }
+  
+  private static Set<String> localIpAddresses = Collections
+      .synchronizedSet(new HashSet<String>());
+  
+  private static boolean isLocalAddress(InetSocketAddress targetAddr) {
+    InetAddress addr = targetAddr.getAddress();
+    if (localIpAddresses.contains(addr.getHostAddress())) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Address " + targetAddr + " is local");
+      }
+      return true;
+    }
+
+    // Check if the address is any local or loop back
+    boolean local = addr.isAnyLocalAddress() || addr.isLoopbackAddress();
+
+    // Check if the address is defined on any interface
+    if (!local) {
+      try {
+        local = NetworkInterface.getByInetAddress(addr) != null;
+      } catch (SocketException e) {
+        local = false;
+      }
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Address " + targetAddr + " is local");
+    }
+    if (local == true) {
+      localIpAddresses.add(addr.getHostAddress());
+    }
+    return local;
+  }
+  
+  /**
+   * Should the block access token be refetched on an exception
+   * 
+   * @param ex Exception received
+   * @param targetAddr Target datanode address from where exception was received
+   * @return true if block access token has expired or invalid and it should be
+   *         refetched
+   */
+  private static boolean tokenRefetchNeeded(IOException ex,
+      InetSocketAddress targetAddr) {
+    /*
+     * Get a new access token and retry. Retry is needed in 2 cases. 1) When
+     * both NN and DN re-started while DFSClient holding a cached access token.
+     * 2) In the case that NN fails to update its access key at pre-set interval
+     * (by a wide margin) and subsequently restarts. In this case, DN
+     * re-registers itself with NN and receives a new access key, but DN will
+     * delete the old access key from its memory since it's considered expired
+     * based on the estimated expiration date.
+     */
+    if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
+      LOG.info("Access token was invalid when connecting to " + targetAddr
+          + " : " + ex);
+      return true;
+    }
+    return false;
+  }
+  
   /**
    * Cancel a delegation token
    * @param token the token to cancel
@@ -1630,7 +1724,7 @@ public class DFSClient implements java.io.Closeable {
     synchronized List<LocatedBlock> getAllBlocks() throws IOException {
       return ((DFSInputStream)in).getAllBlocks();
     }
-
+    
     /**
      * @return The visible length of the file.
      */
@@ -1638,6 +1732,14 @@ public class DFSClient implements java.io.Closeable {
       return ((DFSInputStream)in).getFileLength();
     }
   }
+  
+  boolean shouldTryShortCircuitRead(InetSocketAddress targetAddr)
+      throws IOException {
+    if (shortCircuitLocalReads && isLocalAddress(targetAddr)) {
+      return true;
+    }
+    return false;
+  }
 
   void reportChecksumFailure(String file, ExtendedBlock blk, DatanodeInfo dn) {
     DatanodeInfo [] dnArr = { dn };
@@ -1660,4 +1762,8 @@ public class DFSClient implements java.io.Closeable {
     return getClass().getSimpleName() + "[clientName=" + clientName
         + ", ugi=" + ugi + "]"; 
   }
+
+  void disableShortCircuit() {
+    shortCircuitLocalReads = false;
+  }
 }

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -262,6 +262,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";
   public static final int     DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED = 500;
 
+  public static final String DFS_CLIENT_READ_SHORTCIRCUIT_KEY = "dfs.client.read.shortcircuit";
+  public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT = false;
+  public static final String DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY = "dfs.client.read.shortcircuit.skip.checksum";
+  public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT = false;
+
   // property for fsimage compression
   public static final String DFS_IMAGE_COMPRESS_KEY = "dfs.image.compress";
   public static final boolean DFS_IMAGE_COMPRESS_DEFAULT = false;
@@ -302,6 +307,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY = "dfs.web.authentication.kerberos.principal";
   public static final String  DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY = "dfs.web.authentication.kerberos.keytab";
   
+  public static final String DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY = "dfs.block.local-path-access.user";
+
   // HA related configuration
   public static final String DFS_HA_NAMENODES_KEY = "dfs.ha.namenodes";
   public static final String DFS_HA_NAMENODE_ID_KEY = "dfs.ha.namenode.id";

+ 19 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
 
 /****************************************************************
@@ -405,11 +406,8 @@ public class DFSInputStream extends FSInputStream {
       try {
         ExtendedBlock blk = targetBlock.getBlock();
         Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
-        
-        blockReader = getBlockReader(
-            targetAddr, src, blk,
-            accessToken,
-            offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
+        blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
+            accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
             buffersize, verifyChecksum, dfsClient.clientName);
         if(connectFailedOnce) {
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
@@ -543,7 +541,7 @@ public class DFSInputStream extends FSInputStream {
           if (pos > blockEnd) {
             currentNode = blockSeekTo(pos);
           }
-          int realLen = (int) Math.min((long) len, (blockEnd - pos + 1L));
+          int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
           int result = readBuffer(buf, off, realLen, corruptedBlockMap);
           
           if (result >= 0) {
@@ -666,12 +664,9 @@ public class DFSInputStream extends FSInputStream {
         Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
             
         int len = (int) (end - start + 1);
-
-        reader = getBlockReader(targetAddr, src,
-                                block.getBlock(),
-                                blockToken,
-                                start, len, buffersize,
-                                verifyChecksum, dfsClient.clientName);
+        reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
+            blockToken, start, len, buffersize, verifyChecksum,
+            dfsClient.clientName);
         int nread = reader.readAll(buf, offset, len);
         if (nread != len) {
           throw new IOException("truncated return from reader.read(): " +
@@ -684,6 +679,10 @@ public class DFSInputStream extends FSInputStream {
                  e.getPos() + " from " + chosenNode.getName());
         // we want to remember what we have tried
         addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
+      } catch (AccessControlException ex) {
+        DFSClient.LOG.warn("Short circuit access failed ", ex);
+        dfsClient.disableShortCircuit();
+        continue;
       } catch (IOException e) {
         if (e instanceof InvalidBlockTokenException && refetchToken > 0) {
           DFSClient.LOG.info("Will get a new access token and retry, "
@@ -726,6 +725,7 @@ public class DFSInputStream extends FSInputStream {
    * Otherwise, it will create a new connection.
    *
    * @param dnAddr  Address of the datanode
+   * @param chosenNode Chosen datanode information
    * @param file  File location
    * @param block  The Block object
    * @param blockToken  The access token for security
@@ -737,6 +737,7 @@ public class DFSInputStream extends FSInputStream {
    * @return New BlockReader instance
    */
   protected BlockReader getBlockReader(InetSocketAddress dnAddr,
+                                       DatanodeInfo chosenNode,
                                        String file,
                                        ExtendedBlock block,
                                        Token<BlockTokenIdentifier> blockToken,
@@ -746,6 +747,12 @@ public class DFSInputStream extends FSInputStream {
                                        boolean verifyChecksum,
                                        String clientName)
       throws IOException {
+    
+    if (dfsClient.shouldTryShortCircuitRead(dnAddr)) {
+      return DFSClient.getLocalBlockReader(dfsClient.conf, src, block,
+          blockToken, chosenNode, dfsClient.hdfsTimeout, startOffset);
+    }
+    
     IOException err = null;
     boolean fromCache = true;
 

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -723,6 +724,14 @@ public class DFSUtil {
     return collection != null && collection.size() != 0;
   }
   
+  /** Create {@link ClientDatanodeProtocol} proxy using kerberos ticket */
+  static ClientDatanodeProtocol createClientDatanodeProtocolProxy(
+      DatanodeID datanodeid, Configuration conf, int socketTimeout)
+      throws IOException {
+    return new org.apache.hadoop.hdfs.protocolR23Compatible.ClientDatanodeProtocolTranslatorR23(
+        datanodeid, conf, socketTimeout);
+  }
+  
   /** Create a {@link ClientDatanodeProtocol} proxy */
   public static ClientDatanodeProtocol createClientDatanodeProtocolProxy(
       InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java

@@ -24,7 +24,6 @@ import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.nio.ByteBuffer;
@@ -37,12 +36,9 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java

@@ -85,7 +85,7 @@ public class RemoteBlockReader2  implements BlockReader {
   
   Socket dnSock; //for now just sending the status code (e.g. checksumOk) after the read.
   private ReadableByteChannel in;
-  private DataChecksum checksum;
+  protected DataChecksum checksum;
   
   private PacketHeader curHeader;
   private ByteBuffer curPacketBuf = null;
@@ -96,25 +96,25 @@ public class RemoteBlockReader2  implements BlockReader {
   private long lastSeqNo = -1;
 
   /** offset in block where reader wants to actually read */
-  private long startOffset;
-  private final String filename;
+  protected long startOffset;
+  protected final String filename;
 
-  private static DirectBufferPool bufferPool =
+  protected static DirectBufferPool bufferPool =
     new DirectBufferPool();
   private ByteBuffer headerBuf = ByteBuffer.allocate(
       PacketHeader.PKT_HEADER_LEN);
 
-  private int bytesPerChecksum;
-  private int checksumSize;
+  protected int bytesPerChecksum;
+  protected int checksumSize;
 
   /**
    * The total number of bytes we need to transfer from the DN.
    * This is the amount that the user has requested plus some padding
    * at the beginning so that the read can begin on a chunk boundary.
    */
-  private long bytesNeededToFinish;
+  protected long bytesNeededToFinish;
 
-  private final boolean verifyChecksum;
+  protected final boolean verifyChecksum;
 
   private boolean sentStatusCode = false;
   
@@ -271,7 +271,7 @@ public class RemoteBlockReader2  implements BlockReader {
     }
   }
 
-  private RemoteBlockReader2(String file, String bpid, long blockId,
+  protected RemoteBlockReader2(String file, String bpid, long blockId,
       ReadableByteChannel in, DataChecksum checksum, boolean verifyChecksum,
       long startOffset, long firstChunkOffset, long bytesToRead, Socket dnSock) {
     // Path is used only for printing block and file information in debug

+ 97 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java

@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**
+ * A block and the full path information to the block data file and
+ * the metadata file stored on the local file system.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockLocalPathInfo implements Writable {
+  static final WritableFactory FACTORY = new WritableFactory() {
+    public Writable newInstance() { return new BlockLocalPathInfo(); }
+  };
+  static {                                      // register a ctor
+    WritableFactories.setFactory(BlockLocalPathInfo.class, FACTORY);
+  }
+
+  private ExtendedBlock block;
+  private String localBlockPath = "";  // local file storing the data
+  private String localMetaPath = "";   // local file storing the checksum
+
+  public BlockLocalPathInfo() {}
+
+  /**
+   * Constructs BlockLocalPathInfo.
+   * @param b The block corresponding to this lock path info. 
+   * @param file Block data file.
+   * @param metafile Metadata file for the block.
+   */
+  public BlockLocalPathInfo(ExtendedBlock b, String file, String metafile) {
+    block = b;
+    localBlockPath = file;
+    localMetaPath = metafile;
+  }
+
+  /**
+   * Get the Block data file.
+   * @return Block data file.
+   */
+  public String getBlockPath() {return localBlockPath;}
+  
+  /**
+   * Get the Block metadata file.
+   * @return Block metadata file.
+   */
+  public String getMetaPath() {return localMetaPath;}
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    block.write(out);
+    Text.writeString(out, localBlockPath);
+    Text.writeString(out, localMetaPath);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    block = new ExtendedBlock();
+    block.readFields(in);
+    localBlockPath = Text.readString(in);
+    localMetaPath = Text.readString(in);
+  }
+  
+  /**
+   * Get number of bytes in the block.
+   * @return Number of bytes in the block.
+   */
+  public long getNumBytes() {
+    return block.getNumBytes();
+  }
+}

+ 28 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -19,14 +19,14 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
 
 /** An client-datanode protocol for block recovery
@@ -84,5 +84,30 @@ public interface ClientDatanodeProtocol extends VersionedProtocol {
    *          deleted along with its contents.
    * @throws IOException
    */
-  void deleteBlockPool(String bpid, boolean force) throws IOException; 
+  void deleteBlockPool(String bpid, boolean force) throws IOException;
+  
+  /**
+   * Retrieves the path names of the block file and metadata file stored on the
+   * local file system.
+   * 
+   * In order for this method to work, one of the following should be satisfied:
+   * <ul>
+   * <li>
+   * The client user must be configured at the datanode to be able to use this
+   * method.</li>
+   * <li>
+   * When security is enabled, kerberos authentication must be used to connect
+   * to the datanode.</li>
+   * </ul>
+   * 
+   * @param block
+   *          the specified block on the local datanode
+   * @param token
+   *          the block access token.
+   * @return the BlockLocalPathInfo of a block
+   * @throws IOException
+   *           on error
+   */
+  BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
+      Token<BlockTokenIdentifier> token) throws IOException;
 }

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -852,9 +852,9 @@ public interface ClientProtocol extends VersionedProtocol {
 
   /**
    * Create symlink to a file or directory.
-   * @param target The pathname of the destination that the
+   * @param target The path of the destination that the
    *               link points to.
-   * @param link The pathname of the link being created.
+   * @param link The path of the link being created.
    * @param dirPerm permissions to use when creating parent directories
    * @param createParent - if true then missing parent dirs are created
    *                       if false then parent must exist
@@ -875,14 +875,16 @@ public interface ClientProtocol extends VersionedProtocol {
       IOException;
 
   /**
-   * Resolve the first symbolic link on the specified path.
-   * @param path The pathname that needs to be resolved
-   * 
-   * @return The pathname after resolving the first symbolic link if any.
-   * 
+   * Return the target of the given symlink. If there is an intermediate
+   * symlink in the path (ie a symlink leading up to the final path component)
+   * then the given path is returned with this symlink resolved.
+   *
+   * @param path The path with a link that needs resolution.
+   * @return The path after resolving the first symbolic link in the path.
    * @throws AccessControlException permission denied
    * @throws FileNotFoundException If <code>path</code> does not exist
-   * @throws IOException If an I/O error occurred
+   * @throws IOException If the given path does not refer to a symlink
+   *           or an I/O error occurred
    */
   @Idempotent
   public String getLinkTarget(String path) throws AccessControlException,

+ 25 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java

@@ -32,10 +32,10 @@ import org.apache.hadoop.fs.Path;
 @InterfaceStability.Evolving
 public final class UnresolvedPathException extends UnresolvedLinkException {
   private static final long serialVersionUID = 1L;
-  private String originalPath;  // The original path containing the link
-  private String linkTarget;    // The target of the link 
-  private String remainingPath; // The path part following the link
-  
+  private String path;        // The path containing the link
+  private String preceding;   // The path part preceding the link
+  private String remainder;   // The path part following the link
+  private String linkTarget;  // The link's target
 
   /**
    * Used by RemoteException to instantiate an UnresolvedPathException.
@@ -44,22 +44,30 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
     super(msg);
   }
   
-  public UnresolvedPathException(String originalPath, String remainingPath, 
-      String linkTarget) {
-    this.originalPath  = originalPath;
-    this.remainingPath = remainingPath;
-    this.linkTarget    = linkTarget;
+  public UnresolvedPathException(String path, String preceding,
+      String remainder, String linkTarget) {
+    this.path = path;
+    this.preceding = preceding;
+    this.remainder = remainder;
+    this.linkTarget = linkTarget;
   }
 
-  public Path getUnresolvedPath() throws IOException {
-    return new Path(originalPath);
-  }
-  
+  /**
+   * Return a path with the link resolved with the target.
+   */
   public Path getResolvedPath() throws IOException {
-    if (remainingPath == null || "".equals(remainingPath)) {
-      return new Path(linkTarget);
+    // If the path is absolute we cam throw out the preceding part and
+    // just append the remainder to the target, otherwise append each
+    // piece to resolve the link in path.
+    boolean noRemainder = (remainder == null || "".equals(remainder));
+    Path target = new Path(linkTarget);
+    if (target.isUriPathAbsolute()) {
+      return noRemainder ? target : new Path(target, remainder);
+    } else {
+      return noRemainder
+        ? new Path(preceding, target)
+        : new Path(new Path(preceding, linkTarget), remainder);
     }
-    return new Path(linkTarget, remainingPath);
   }
 
   @Override
@@ -68,7 +76,7 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
     if (msg != null) {
       return msg;
     }
-    String myMsg = "Unresolved path " + originalPath;
+    String myMsg = "Unresolved path " + path;
     try {
       return getResolvedPath().toString();
     } catch (IOException e) {

文件差异内容过多而无法显示
+ 11652 - 1124
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/HdfsProtos.java


+ 2516 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/InterDatanodeProtocolProtos.java

@@ -0,0 +1,2516 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: InterDatanodeProtocol.proto
+
+package org.apache.hadoop.hdfs.protocol.proto;
+
+public final class InterDatanodeProtocolProtos {
+  private InterDatanodeProtocolProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface InitReplicaRecoveryRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .RecoveringBlockProto block = 1;
+    boolean hasBlock();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto getBlock();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder getBlockOrBuilder();
+  }
+  public static final class InitReplicaRecoveryRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements InitReplicaRecoveryRequestProtoOrBuilder {
+    // Use InitReplicaRecoveryRequestProto.newBuilder() to construct.
+    private InitReplicaRecoveryRequestProto(Builder builder) {
+      super(builder);
+    }
+    private InitReplicaRecoveryRequestProto(boolean noInit) {}
+    
+    private static final InitReplicaRecoveryRequestProto defaultInstance;
+    public static InitReplicaRecoveryRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public InitReplicaRecoveryRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_InitReplicaRecoveryRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_InitReplicaRecoveryRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .RecoveringBlockProto block = 1;
+    public static final int BLOCK_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto block_;
+    public boolean hasBlock() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto getBlock() {
+      return block_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder getBlockOrBuilder() {
+      return block_;
+    }
+    
+    private void initFields() {
+      block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getBlock().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      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_);
+      }
+      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_);
+      }
+      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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlock() == other.hasBlock());
+      if (hasBlock()) {
+        result = result && getBlock()
+            .equals(other.getBlock());
+      }
+      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();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_InitReplicaRecoveryRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_InitReplicaRecoveryRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto.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.RecoveringBlockProto.getDefaultInstance();
+        } else {
+          blockBuilder_.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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto(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();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto.getDefaultInstance()) return this;
+        if (other.hasBlock()) {
+          mergeBlock(other.getBlock());
+        }
+        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.RecoveringBlockProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.newBuilder();
+              if (hasBlock()) {
+                subBuilder.mergeFrom(getBlock());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setBlock(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .RecoveringBlockProto block = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          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> blockBuilder_;
+      public boolean hasBlock() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto getBlock() {
+        if (blockBuilder_ == null) {
+          return block_;
+        } else {
+          return blockBuilder_.getMessage();
+        }
+      }
+      public Builder setBlock(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto 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.RecoveringBlockProto.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.RecoveringBlockProto value) {
+        if (blockBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              block_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.getDefaultInstance()) {
+            block_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.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.RecoveringBlockProto.getDefaultInstance();
+          onChanged();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto.Builder getBlockBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getBlockFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProtoOrBuilder getBlockOrBuilder() {
+        if (blockBuilder_ != null) {
+          return blockBuilder_.getMessageOrBuilder();
+        } else {
+          return block_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          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> 
+          getBlockFieldBuilder() {
+        if (blockBuilder_ == null) {
+          blockBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              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>(
+                  block_,
+                  getParentForChildren(),
+                  isClean());
+          block_ = null;
+        }
+        return blockBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:InitReplicaRecoveryRequestProto)
+    }
+    
+    static {
+      defaultInstance = new InitReplicaRecoveryRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:InitReplicaRecoveryRequestProto)
+  }
+  
+  public interface InitReplicaRecoveryResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .ReplicaState state = 1;
+    boolean hasState();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState getState();
+    
+    // required .BlockProto block = 2;
+    boolean hasBlock();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto getBlock();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder getBlockOrBuilder();
+  }
+  public static final class InitReplicaRecoveryResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements InitReplicaRecoveryResponseProtoOrBuilder {
+    // Use InitReplicaRecoveryResponseProto.newBuilder() to construct.
+    private InitReplicaRecoveryResponseProto(Builder builder) {
+      super(builder);
+    }
+    private InitReplicaRecoveryResponseProto(boolean noInit) {}
+    
+    private static final InitReplicaRecoveryResponseProto defaultInstance;
+    public static InitReplicaRecoveryResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public InitReplicaRecoveryResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_InitReplicaRecoveryResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_InitReplicaRecoveryResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .ReplicaState state = 1;
+    public static final int STATE_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState state_;
+    public boolean hasState() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState getState() {
+      return state_;
+    }
+    
+    // required .BlockProto block = 2;
+    public static final int BLOCK_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto block_;
+    public boolean hasBlock() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto getBlock() {
+      return block_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder getBlockOrBuilder() {
+      return block_;
+    }
+    
+    private void initFields() {
+      state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState.FINALIZED;
+      block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasState()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      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.writeEnum(1, state_.getNumber());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, block_);
+      }
+      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, state_.getNumber());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, block_);
+      }
+      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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasState() == other.hasState());
+      if (hasState()) {
+        result = result &&
+            (getState() == other.getState());
+      }
+      result = result && (hasBlock() == other.hasBlock());
+      if (hasBlock()) {
+        result = result && getBlock()
+            .equals(other.getBlock());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasState()) {
+        hash = (37 * hash) + STATE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getState());
+      }
+      if (hasBlock()) {
+        hash = (37 * hash) + BLOCK_FIELD_NUMBER;
+        hash = (53 * hash) + getBlock().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_InitReplicaRecoveryResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_InitReplicaRecoveryResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.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();
+        state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState.FINALIZED;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (blockBuilder_ == null) {
+          block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.state_ = state_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (blockBuilder_ == null) {
+          result.block_ = block_;
+        } else {
+          result.block_ = blockBuilder_.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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.getDefaultInstance()) return this;
+        if (other.hasState()) {
+          setState(other.getState());
+        }
+        if (other.hasBlock()) {
+          mergeBlock(other.getBlock());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasState()) {
+          
+          return false;
+        }
+        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 8: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState value = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                state_ = value;
+              }
+              break;
+            }
+            case 18: {
+              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;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .ReplicaState state = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState.FINALIZED;
+      public boolean hasState() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState getState() {
+        return state_;
+      }
+      public Builder setState(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        state_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearState() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState.FINALIZED;
+        onChanged();
+        return this;
+      }
+      
+      // required .BlockProto block = 2;
+      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_ & 0x00000002) == 0x00000002);
+      }
+      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_ |= 0x00000002;
+        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_ |= 0x00000002;
+        return this;
+      }
+      public Builder mergeBlock(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto value) {
+        if (blockBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              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_ |= 0x00000002;
+        return this;
+      }
+      public Builder clearBlock() {
+        if (blockBuilder_ == null) {
+          block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance();
+          onChanged();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.Builder getBlockBuilder() {
+        bitField0_ |= 0x00000002;
+        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_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:InitReplicaRecoveryResponseProto)
+    }
+    
+    static {
+      defaultInstance = new InitReplicaRecoveryResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:InitReplicaRecoveryResponseProto)
+  }
+  
+  public interface UpdateReplicaUnderRecoveryRequestProtoOrBuilder
+      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 recoveryId = 2;
+    boolean hasRecoveryId();
+    long getRecoveryId();
+    
+    // required uint64 newLength = 3;
+    boolean hasNewLength();
+    long getNewLength();
+  }
+  public static final class UpdateReplicaUnderRecoveryRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements UpdateReplicaUnderRecoveryRequestProtoOrBuilder {
+    // Use UpdateReplicaUnderRecoveryRequestProto.newBuilder() to construct.
+    private UpdateReplicaUnderRecoveryRequestProto(Builder builder) {
+      super(builder);
+    }
+    private UpdateReplicaUnderRecoveryRequestProto(boolean noInit) {}
+    
+    private static final UpdateReplicaUnderRecoveryRequestProto defaultInstance;
+    public static UpdateReplicaUnderRecoveryRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public UpdateReplicaUnderRecoveryRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_UpdateReplicaUnderRecoveryRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_UpdateReplicaUnderRecoveryRequestProto_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 recoveryId = 2;
+    public static final int RECOVERYID_FIELD_NUMBER = 2;
+    private long recoveryId_;
+    public boolean hasRecoveryId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getRecoveryId() {
+      return recoveryId_;
+    }
+    
+    // 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_;
+    }
+    
+    private void initFields() {
+      block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+      recoveryId_ = 0L;
+      newLength_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasRecoveryId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasNewLength()) {
+        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.writeUInt64(2, recoveryId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt64(3, newLength_);
+      }
+      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, recoveryId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, newLength_);
+      }
+      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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlock() == other.hasBlock());
+      if (hasBlock()) {
+        result = result && getBlock()
+            .equals(other.getBlock());
+      }
+      result = result && (hasRecoveryId() == other.hasRecoveryId());
+      if (hasRecoveryId()) {
+        result = result && (getRecoveryId()
+            == other.getRecoveryId());
+      }
+      result = result && (hasNewLength() == other.hasNewLength());
+      if (hasNewLength()) {
+        result = result && (getNewLength()
+            == other.getNewLength());
+      }
+      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 (hasRecoveryId()) {
+        hash = (37 * hash) + RECOVERYID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getRecoveryId());
+      }
+      if (hasNewLength()) {
+        hash = (37 * hash) + NEWLENGTH_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getNewLength());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_UpdateReplicaUnderRecoveryRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_UpdateReplicaUnderRecoveryRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto.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.ExtendedBlockProto.getDefaultInstance();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        recoveryId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        newLength_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto(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.recoveryId_ = recoveryId_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.newLength_ = newLength_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto.getDefaultInstance()) return this;
+        if (other.hasBlock()) {
+          mergeBlock(other.getBlock());
+        }
+        if (other.hasRecoveryId()) {
+          setRecoveryId(other.getRecoveryId());
+        }
+        if (other.hasNewLength()) {
+          setNewLength(other.getNewLength());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBlock()) {
+          
+          return false;
+        }
+        if (!hasRecoveryId()) {
+          
+          return false;
+        }
+        if (!hasNewLength()) {
+          
+          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.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;
+              recoveryId_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              newLength_ = input.readUInt64();
+              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 recoveryId = 2;
+      private long recoveryId_ ;
+      public boolean hasRecoveryId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getRecoveryId() {
+        return recoveryId_;
+      }
+      public Builder setRecoveryId(long value) {
+        bitField0_ |= 0x00000002;
+        recoveryId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearRecoveryId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        recoveryId_ = 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;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:UpdateReplicaUnderRecoveryRequestProto)
+    }
+    
+    static {
+      defaultInstance = new UpdateReplicaUnderRecoveryRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:UpdateReplicaUnderRecoveryRequestProto)
+  }
+  
+  public interface UpdateReplicaUnderRecoveryResponseProtoOrBuilder
+      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();
+  }
+  public static final class UpdateReplicaUnderRecoveryResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements UpdateReplicaUnderRecoveryResponseProtoOrBuilder {
+    // Use UpdateReplicaUnderRecoveryResponseProto.newBuilder() to construct.
+    private UpdateReplicaUnderRecoveryResponseProto(Builder builder) {
+      super(builder);
+    }
+    private UpdateReplicaUnderRecoveryResponseProto(boolean noInit) {}
+    
+    private static final UpdateReplicaUnderRecoveryResponseProto defaultInstance;
+    public static UpdateReplicaUnderRecoveryResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public UpdateReplicaUnderRecoveryResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_UpdateReplicaUnderRecoveryResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_UpdateReplicaUnderRecoveryResponseProto_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_;
+    }
+    
+    private void initFields() {
+      block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getBlock().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      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_);
+      }
+      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_);
+      }
+      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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlock() == other.hasBlock());
+      if (hasBlock()) {
+        result = result && getBlock()
+            .equals(other.getBlock());
+      }
+      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();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto 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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_UpdateReplicaUnderRecoveryResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.internal_static_UpdateReplicaUnderRecoveryResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.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.ExtendedBlockProto.getDefaultInstance();
+        } else {
+          blockBuilder_.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.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto(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();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.getDefaultInstance()) return this;
+        if (other.hasBlock()) {
+          mergeBlock(other.getBlock());
+        }
+        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.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;
+            }
+          }
+        }
+      }
+      
+      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_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:UpdateReplicaUnderRecoveryResponseProto)
+    }
+    
+    static {
+      defaultInstance = new UpdateReplicaUnderRecoveryResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:UpdateReplicaUnderRecoveryResponseProto)
+  }
+  
+  public static abstract class InterDatanodeProtocolService
+      implements com.google.protobuf.Service {
+    protected InterDatanodeProtocolService() {}
+    
+    public interface Interface {
+      public abstract void initReplicaRecovery(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto> done);
+      
+      public abstract void updateReplicaUnderRecovery(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto> done);
+      
+    }
+    
+    public static com.google.protobuf.Service newReflectiveService(
+        final Interface impl) {
+      return new InterDatanodeProtocolService() {
+        @java.lang.Override
+        public  void initReplicaRecovery(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto> done) {
+          impl.initReplicaRecovery(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void updateReplicaUnderRecovery(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto> done) {
+          impl.updateReplicaUnderRecovery(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.initReplicaRecovery(controller, (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto)request);
+            case 1:
+              return impl.updateReplicaUnderRecovery(controller, (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto)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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto.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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+      };
+    }
+    
+    public abstract void initReplicaRecovery(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto> done);
+    
+    public abstract void updateReplicaUnderRecovery(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto> done);
+    
+    public static final
+        com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.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.initReplicaRecovery(controller, (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto>specializeCallback(
+              done));
+          return;
+        case 1:
+          this.updateReplicaUnderRecovery(controller, (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto>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.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto.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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.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.InterDatanodeProtocolProtos.InterDatanodeProtocolService 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 initReplicaRecovery(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.getDefaultInstance()));
+      }
+      
+      public  void updateReplicaUnderRecovery(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.getDefaultInstance()));
+      }
+    }
+    
+    public static BlockingInterface newBlockingStub(
+        com.google.protobuf.BlockingRpcChannel channel) {
+      return new BlockingStub(channel);
+    }
+    
+    public interface BlockingInterface {
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto initReplicaRecovery(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto updateReplicaUnderRecovery(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto 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.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto initReplicaRecovery(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto updateReplicaUnderRecovery(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.getDefaultInstance());
+      }
+      
+    }
+  }
+  
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_InitReplicaRecoveryRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_InitReplicaRecoveryRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_InitReplicaRecoveryResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_InitReplicaRecoveryResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_UpdateReplicaUnderRecoveryRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_UpdateReplicaUnderRecoveryRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_UpdateReplicaUnderRecoveryResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_UpdateReplicaUnderRecoveryResponseProto_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\033InterDatanodeProtocol.proto\032\nhdfs.prot" +
+      "o\"G\n\037InitReplicaRecoveryRequestProto\022$\n\005" +
+      "block\030\001 \002(\0132\025.RecoveringBlockProto\"\\\n In" +
+      "itReplicaRecoveryResponseProto\022\034\n\005state\030" +
+      "\001 \002(\0162\r.ReplicaState\022\032\n\005block\030\002 \002(\0132\013.Bl" +
+      "ockProto\"s\n&UpdateReplicaUnderRecoveryRe" +
+      "questProto\022\"\n\005block\030\001 \002(\0132\023.ExtendedBloc" +
+      "kProto\022\022\n\nrecoveryId\030\002 \002(\004\022\021\n\tnewLength\030" +
+      "\003 \002(\004\"M\n\'UpdateReplicaUnderRecoveryRespo" +
+      "nseProto\022\"\n\005block\030\001 \002(\0132\023.ExtendedBlockP",
+      "roto2\353\001\n\034InterDatanodeProtocolService\022Z\n" +
+      "\023initReplicaRecovery\022 .InitReplicaRecove" +
+      "ryRequestProto\032!.InitReplicaRecoveryResp" +
+      "onseProto\022o\n\032updateReplicaUnderRecovery\022" +
+      "\'.UpdateReplicaUnderRecoveryRequestProto" +
+      "\032(.UpdateReplicaUnderRecoveryResponsePro" +
+      "toBJ\n%org.apache.hadoop.hdfs.protocol.pr" +
+      "otoB\033InterDatanodeProtocolProtos\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_InitReplicaRecoveryRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_InitReplicaRecoveryRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_InitReplicaRecoveryRequestProto_descriptor,
+              new java.lang.String[] { "Block", },
+              org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto.Builder.class);
+          internal_static_InitReplicaRecoveryResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_InitReplicaRecoveryResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_InitReplicaRecoveryResponseProto_descriptor,
+              new java.lang.String[] { "State", "Block", },
+              org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto.Builder.class);
+          internal_static_UpdateReplicaUnderRecoveryRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(2);
+          internal_static_UpdateReplicaUnderRecoveryRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_UpdateReplicaUnderRecoveryRequestProto_descriptor,
+              new java.lang.String[] { "Block", "RecoveryId", "NewLength", },
+              org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto.Builder.class);
+          internal_static_UpdateReplicaUnderRecoveryResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(3);
+          internal_static_UpdateReplicaUnderRecoveryResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_UpdateReplicaUnderRecoveryResponseProto_descriptor,
+              new java.lang.String[] { "Block", },
+              org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto.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)
+}

+ 2234 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/JournalProtocolProtos.java

@@ -0,0 +1,2234 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: JournalProtocol.proto
+
+package org.apache.hadoop.hdfs.protocol.proto;
+
+public final class JournalProtocolProtos {
+  private JournalProtocolProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface JournalRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .NamenodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+    
+    // required uint64 firstTxnId = 2;
+    boolean hasFirstTxnId();
+    long getFirstTxnId();
+    
+    // required uint32 numTxns = 3;
+    boolean hasNumTxns();
+    int getNumTxns();
+    
+    // required bytes records = 4;
+    boolean hasRecords();
+    com.google.protobuf.ByteString getRecords();
+  }
+  public static final class JournalRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements JournalRequestProtoOrBuilder {
+    // Use JournalRequestProto.newBuilder() to construct.
+    private JournalRequestProto(Builder builder) {
+      super(builder);
+    }
+    private JournalRequestProto(boolean noInit) {}
+    
+    private static final JournalRequestProto defaultInstance;
+    public static JournalRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public JournalRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_JournalRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_JournalRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .NamenodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    // required uint64 firstTxnId = 2;
+    public static final int FIRSTTXNID_FIELD_NUMBER = 2;
+    private long firstTxnId_;
+    public boolean hasFirstTxnId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getFirstTxnId() {
+      return firstTxnId_;
+    }
+    
+    // required uint32 numTxns = 3;
+    public static final int NUMTXNS_FIELD_NUMBER = 3;
+    private int numTxns_;
+    public boolean hasNumTxns() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public int getNumTxns() {
+      return numTxns_;
+    }
+    
+    // required bytes records = 4;
+    public static final int RECORDS_FIELD_NUMBER = 4;
+    private com.google.protobuf.ByteString records_;
+    public boolean hasRecords() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    public com.google.protobuf.ByteString getRecords() {
+      return records_;
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      firstTxnId_ = 0L;
+      numTxns_ = 0;
+      records_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasRegistration()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasFirstTxnId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasNumTxns()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasRecords()) {
+        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, firstTxnId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt32(3, numTxns_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBytes(4, records_);
+      }
+      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, firstTxnId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(3, numTxns_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(4, records_);
+      }
+      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.JournalProtocolProtos.JournalRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasRegistration() == other.hasRegistration());
+      if (hasRegistration()) {
+        result = result && getRegistration()
+            .equals(other.getRegistration());
+      }
+      result = result && (hasFirstTxnId() == other.hasFirstTxnId());
+      if (hasFirstTxnId()) {
+        result = result && (getFirstTxnId()
+            == other.getFirstTxnId());
+      }
+      result = result && (hasNumTxns() == other.hasNumTxns());
+      if (hasNumTxns()) {
+        result = result && (getNumTxns()
+            == other.getNumTxns());
+      }
+      result = result && (hasRecords() == other.hasRecords());
+      if (hasRecords()) {
+        result = result && getRecords()
+            .equals(other.getRecords());
+      }
+      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 (hasFirstTxnId()) {
+        hash = (37 * hash) + FIRSTTXNID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getFirstTxnId());
+      }
+      if (hasNumTxns()) {
+        hash = (37 * hash) + NUMTXNS_FIELD_NUMBER;
+        hash = (53 * hash) + getNumTxns();
+      }
+      if (hasRecords()) {
+        hash = (37 * hash) + RECORDS_FIELD_NUMBER;
+        hash = (53 * hash) + getRecords().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto 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.JournalProtocolProtos.JournalRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto 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.JournalProtocolProtos.JournalRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto 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.JournalProtocolProtos.JournalRequestProto 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.JournalProtocolProtos.JournalRequestProto 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.JournalProtocolProtos.JournalRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto 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.JournalProtocolProtos.JournalRequestProto 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.JournalProtocolProtos.JournalRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_JournalRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_JournalRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto.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.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        firstTxnId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        numTxns_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        records_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto(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.firstTxnId_ = firstTxnId_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.numTxns_ = numTxns_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.records_ = records_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto.getDefaultInstance()) return this;
+        if (other.hasRegistration()) {
+          mergeRegistration(other.getRegistration());
+        }
+        if (other.hasFirstTxnId()) {
+          setFirstTxnId(other.getFirstTxnId());
+        }
+        if (other.hasNumTxns()) {
+          setNumTxns(other.getNumTxns());
+        }
+        if (other.hasRecords()) {
+          setRecords(other.getRecords());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasRegistration()) {
+          
+          return false;
+        }
+        if (!hasFirstTxnId()) {
+          
+          return false;
+        }
+        if (!hasNumTxns()) {
+          
+          return false;
+        }
+        if (!hasRecords()) {
+          
+          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.HdfsProtos.NamenodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              firstTxnId_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              numTxns_ = input.readUInt32();
+              break;
+            }
+            case 34: {
+              bitField0_ |= 0x00000008;
+              records_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .NamenodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto 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.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // required uint64 firstTxnId = 2;
+      private long firstTxnId_ ;
+      public boolean hasFirstTxnId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getFirstTxnId() {
+        return firstTxnId_;
+      }
+      public Builder setFirstTxnId(long value) {
+        bitField0_ |= 0x00000002;
+        firstTxnId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearFirstTxnId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        firstTxnId_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required uint32 numTxns = 3;
+      private int numTxns_ ;
+      public boolean hasNumTxns() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public int getNumTxns() {
+        return numTxns_;
+      }
+      public Builder setNumTxns(int value) {
+        bitField0_ |= 0x00000004;
+        numTxns_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearNumTxns() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        numTxns_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // required bytes records = 4;
+      private com.google.protobuf.ByteString records_ = com.google.protobuf.ByteString.EMPTY;
+      public boolean hasRecords() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      public com.google.protobuf.ByteString getRecords() {
+        return records_;
+      }
+      public Builder setRecords(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        records_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearRecords() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        records_ = getDefaultInstance().getRecords();
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:JournalRequestProto)
+    }
+    
+    static {
+      defaultInstance = new JournalRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:JournalRequestProto)
+  }
+  
+  public interface JournalResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class JournalResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements JournalResponseProtoOrBuilder {
+    // Use JournalResponseProto.newBuilder() to construct.
+    private JournalResponseProto(Builder builder) {
+      super(builder);
+    }
+    private JournalResponseProto(boolean noInit) {}
+    
+    private static final JournalResponseProto defaultInstance;
+    public static JournalResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public JournalResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_JournalResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_JournalResponseProto_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.JournalProtocolProtos.JournalResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto) 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.JournalProtocolProtos.JournalResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto 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.JournalProtocolProtos.JournalResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto 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.JournalProtocolProtos.JournalResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto 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.JournalProtocolProtos.JournalResponseProto 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.JournalProtocolProtos.JournalResponseProto 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.JournalProtocolProtos.JournalResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto 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.JournalProtocolProtos.JournalResponseProto 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.JournalProtocolProtos.JournalResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_JournalResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_JournalResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto.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.JournalProtocolProtos.JournalResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto.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:JournalResponseProto)
+    }
+    
+    static {
+      defaultInstance = new JournalResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:JournalResponseProto)
+  }
+  
+  public interface StartLogSegmentRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .NamenodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+    
+    // required uint64 txid = 2;
+    boolean hasTxid();
+    long getTxid();
+  }
+  public static final class StartLogSegmentRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements StartLogSegmentRequestProtoOrBuilder {
+    // Use StartLogSegmentRequestProto.newBuilder() to construct.
+    private StartLogSegmentRequestProto(Builder builder) {
+      super(builder);
+    }
+    private StartLogSegmentRequestProto(boolean noInit) {}
+    
+    private static final StartLogSegmentRequestProto defaultInstance;
+    public static StartLogSegmentRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public StartLogSegmentRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_StartLogSegmentRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_StartLogSegmentRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .NamenodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    // required uint64 txid = 2;
+    public static final int TXID_FIELD_NUMBER = 2;
+    private long txid_;
+    public boolean hasTxid() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getTxid() {
+      return txid_;
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      txid_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasRegistration()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTxid()) {
+        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, txid_);
+      }
+      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, txid_);
+      }
+      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.JournalProtocolProtos.StartLogSegmentRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasRegistration() == other.hasRegistration());
+      if (hasRegistration()) {
+        result = result && getRegistration()
+            .equals(other.getRegistration());
+      }
+      result = result && (hasTxid() == other.hasTxid());
+      if (hasTxid()) {
+        result = result && (getTxid()
+            == other.getTxid());
+      }
+      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 (hasTxid()) {
+        hash = (37 * hash) + TXID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTxid());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto 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.JournalProtocolProtos.StartLogSegmentRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto 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.JournalProtocolProtos.StartLogSegmentRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto 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.JournalProtocolProtos.StartLogSegmentRequestProto 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.JournalProtocolProtos.StartLogSegmentRequestProto 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.JournalProtocolProtos.StartLogSegmentRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto 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.JournalProtocolProtos.StartLogSegmentRequestProto 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.JournalProtocolProtos.StartLogSegmentRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_StartLogSegmentRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_StartLogSegmentRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto.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.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        txid_ = 0L;
+        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.JournalProtocolProtos.StartLogSegmentRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto(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.txid_ = txid_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto.getDefaultInstance()) return this;
+        if (other.hasRegistration()) {
+          mergeRegistration(other.getRegistration());
+        }
+        if (other.hasTxid()) {
+          setTxid(other.getTxid());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasRegistration()) {
+          
+          return false;
+        }
+        if (!hasTxid()) {
+          
+          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.HdfsProtos.NamenodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              txid_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .NamenodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto 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.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // required uint64 txid = 2;
+      private long txid_ ;
+      public boolean hasTxid() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getTxid() {
+        return txid_;
+      }
+      public Builder setTxid(long value) {
+        bitField0_ |= 0x00000002;
+        txid_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearTxid() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        txid_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:StartLogSegmentRequestProto)
+    }
+    
+    static {
+      defaultInstance = new StartLogSegmentRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:StartLogSegmentRequestProto)
+  }
+  
+  public interface StartLogSegmentResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class StartLogSegmentResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements StartLogSegmentResponseProtoOrBuilder {
+    // Use StartLogSegmentResponseProto.newBuilder() to construct.
+    private StartLogSegmentResponseProto(Builder builder) {
+      super(builder);
+    }
+    private StartLogSegmentResponseProto(boolean noInit) {}
+    
+    private static final StartLogSegmentResponseProto defaultInstance;
+    public static StartLogSegmentResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public StartLogSegmentResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_StartLogSegmentResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_StartLogSegmentResponseProto_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.JournalProtocolProtos.StartLogSegmentResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto) 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.JournalProtocolProtos.StartLogSegmentResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto 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.JournalProtocolProtos.StartLogSegmentResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto 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.JournalProtocolProtos.StartLogSegmentResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto 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.JournalProtocolProtos.StartLogSegmentResponseProto 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.JournalProtocolProtos.StartLogSegmentResponseProto 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.JournalProtocolProtos.StartLogSegmentResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto 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.JournalProtocolProtos.StartLogSegmentResponseProto 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.JournalProtocolProtos.StartLogSegmentResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_StartLogSegmentResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.internal_static_StartLogSegmentResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.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.JournalProtocolProtos.StartLogSegmentResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.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:StartLogSegmentResponseProto)
+    }
+    
+    static {
+      defaultInstance = new StartLogSegmentResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:StartLogSegmentResponseProto)
+  }
+  
+  public static abstract class JournalProtocolService
+      implements com.google.protobuf.Service {
+    protected JournalProtocolService() {}
+    
+    public interface Interface {
+      public abstract void journal(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto> done);
+      
+      public abstract void startLogSegment(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto> done);
+      
+    }
+    
+    public static com.google.protobuf.Service newReflectiveService(
+        final Interface impl) {
+      return new JournalProtocolService() {
+        @java.lang.Override
+        public  void journal(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto> done) {
+          impl.journal(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void startLogSegment(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto> done) {
+          impl.startLogSegment(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.journal(controller, (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto)request);
+            case 1:
+              return impl.startLogSegment(controller, (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto)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.JournalProtocolProtos.JournalRequestProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto.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.JournalProtocolProtos.JournalResponseProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+      };
+    }
+    
+    public abstract void journal(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto> done);
+    
+    public abstract void startLogSegment(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto> done);
+    
+    public static final
+        com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.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.journal(controller, (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto>specializeCallback(
+              done));
+          return;
+        case 1:
+          this.startLogSegment(controller, (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto>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.JournalProtocolProtos.JournalRequestProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto.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.JournalProtocolProtos.JournalResponseProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.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.JournalProtocolProtos.JournalProtocolService 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 journal(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto.getDefaultInstance()));
+      }
+      
+      public  void startLogSegment(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.getDefaultInstance()));
+      }
+    }
+    
+    public static BlockingInterface newBlockingStub(
+        com.google.protobuf.BlockingRpcChannel channel) {
+      return new BlockingStub(channel);
+    }
+    
+    public interface BlockingInterface {
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto journal(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto startLogSegment(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto 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.JournalProtocolProtos.JournalResponseProto journal(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto startLogSegment(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.getDefaultInstance());
+      }
+      
+    }
+  }
+  
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_JournalRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_JournalRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_JournalResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_JournalResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_StartLogSegmentRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_StartLogSegmentRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_StartLogSegmentResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_StartLogSegmentResponseProto_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\025JournalProtocol.proto\032\nhdfs.proto\"}\n\023J" +
+      "ournalRequestProto\0220\n\014registration\030\001 \002(\013" +
+      "2\032.NamenodeRegistrationProto\022\022\n\nfirstTxn" +
+      "Id\030\002 \002(\004\022\017\n\007numTxns\030\003 \002(\r\022\017\n\007records\030\004 \002" +
+      "(\014\"\026\n\024JournalResponseProto\"]\n\033StartLogSe" +
+      "gmentRequestProto\0220\n\014registration\030\001 \002(\0132" +
+      "\032.NamenodeRegistrationProto\022\014\n\004txid\030\002 \002(" +
+      "\004\"\036\n\034StartLogSegmentResponseProto2\240\001\n\026Jo" +
+      "urnalProtocolService\0226\n\007journal\022\024.Journa" +
+      "lRequestProto\032\025.JournalResponseProto\022N\n\017",
+      "startLogSegment\022\034.StartLogSegmentRequest" +
+      "Proto\032\035.StartLogSegmentResponseProtoBD\n%" +
+      "org.apache.hadoop.hdfs.protocol.protoB\025J" +
+      "ournalProtocolProtos\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_JournalRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_JournalRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_JournalRequestProto_descriptor,
+              new java.lang.String[] { "Registration", "FirstTxnId", "NumTxns", "Records", },
+              org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto.Builder.class);
+          internal_static_JournalResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_JournalResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_JournalResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto.Builder.class);
+          internal_static_StartLogSegmentRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(2);
+          internal_static_StartLogSegmentRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_StartLogSegmentRequestProto_descriptor,
+              new java.lang.String[] { "Registration", "Txid", },
+              org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto.Builder.class);
+          internal_static_StartLogSegmentResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(3);
+          internal_static_StartLogSegmentResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_StartLogSegmentResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto.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)
+}

+ 8949 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/NamenodeProtocolProtos.java

@@ -0,0 +1,8949 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NamenodeProtocol.proto
+
+package org.apache.hadoop.hdfs.protocol.proto;
+
+public final class NamenodeProtocolProtos {
+  private NamenodeProtocolProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface GetBlocksRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeIDProto datanode = 1;
+    boolean hasDatanode();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDatanode();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getDatanodeOrBuilder();
+    
+    // required uint64 size = 2;
+    boolean hasSize();
+    long getSize();
+  }
+  public static final class GetBlocksRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetBlocksRequestProtoOrBuilder {
+    // Use GetBlocksRequestProto.newBuilder() to construct.
+    private GetBlocksRequestProto(Builder builder) {
+      super(builder);
+    }
+    private GetBlocksRequestProto(boolean noInit) {}
+    
+    private static final GetBlocksRequestProto defaultInstance;
+    public static GetBlocksRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetBlocksRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlocksRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlocksRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .DatanodeIDProto datanode = 1;
+    public static final int DATANODE_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto datanode_;
+    public boolean hasDatanode() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDatanode() {
+      return datanode_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getDatanodeOrBuilder() {
+      return datanode_;
+    }
+    
+    // required uint64 size = 2;
+    public static final int SIZE_FIELD_NUMBER = 2;
+    private long size_;
+    public boolean hasSize() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getSize() {
+      return size_;
+    }
+    
+    private void initFields() {
+      datanode_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+      size_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasDatanode()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSize()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getDatanode().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, datanode_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, size_);
+      }
+      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, datanode_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, size_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasDatanode() == other.hasDatanode());
+      if (hasDatanode()) {
+        result = result && getDatanode()
+            .equals(other.getDatanode());
+      }
+      result = result && (hasSize() == other.hasSize());
+      if (hasSize()) {
+        result = result && (getSize()
+            == other.getSize());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasDatanode()) {
+        hash = (37 * hash) + DATANODE_FIELD_NUMBER;
+        hash = (53 * hash) + getDatanode().hashCode();
+      }
+      if (hasSize()) {
+        hash = (37 * hash) + SIZE_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getSize());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto 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.NamenodeProtocolProtos.GetBlocksRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto 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.NamenodeProtocolProtos.GetBlocksRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto 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.NamenodeProtocolProtos.GetBlocksRequestProto 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.NamenodeProtocolProtos.GetBlocksRequestProto 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.NamenodeProtocolProtos.GetBlocksRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto 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.NamenodeProtocolProtos.GetBlocksRequestProto 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.NamenodeProtocolProtos.GetBlocksRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlocksRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlocksRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getDatanodeFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (datanodeBuilder_ == null) {
+          datanode_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+        } else {
+          datanodeBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        size_ = 0L;
+        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.NamenodeProtocolProtos.GetBlocksRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (datanodeBuilder_ == null) {
+          result.datanode_ = datanode_;
+        } else {
+          result.datanode_ = datanodeBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.size_ = size_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto.getDefaultInstance()) return this;
+        if (other.hasDatanode()) {
+          mergeDatanode(other.getDatanode());
+        }
+        if (other.hasSize()) {
+          setSize(other.getSize());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasDatanode()) {
+          
+          return false;
+        }
+        if (!hasSize()) {
+          
+          return false;
+        }
+        if (!getDatanode().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 (hasDatanode()) {
+                subBuilder.mergeFrom(getDatanode());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setDatanode(subBuilder.buildPartial());
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              size_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeIDProto datanode = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto datanode_ = 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> datanodeBuilder_;
+      public boolean hasDatanode() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDatanode() {
+        if (datanodeBuilder_ == null) {
+          return datanode_;
+        } else {
+          return datanodeBuilder_.getMessage();
+        }
+      }
+      public Builder setDatanode(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
+        if (datanodeBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          datanode_ = value;
+          onChanged();
+        } else {
+          datanodeBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setDatanode(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder builderForValue) {
+        if (datanodeBuilder_ == null) {
+          datanode_ = builderForValue.build();
+          onChanged();
+        } else {
+          datanodeBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeDatanode(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
+        if (datanodeBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              datanode_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance()) {
+            datanode_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.newBuilder(datanode_).mergeFrom(value).buildPartial();
+          } else {
+            datanode_ = value;
+          }
+          onChanged();
+        } else {
+          datanodeBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearDatanode() {
+        if (datanodeBuilder_ == null) {
+          datanode_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+          onChanged();
+        } else {
+          datanodeBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder getDatanodeBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getDatanodeFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getDatanodeOrBuilder() {
+        if (datanodeBuilder_ != null) {
+          return datanodeBuilder_.getMessageOrBuilder();
+        } else {
+          return datanode_;
+        }
+      }
+      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> 
+          getDatanodeFieldBuilder() {
+        if (datanodeBuilder_ == null) {
+          datanodeBuilder_ = 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>(
+                  datanode_,
+                  getParentForChildren(),
+                  isClean());
+          datanode_ = null;
+        }
+        return datanodeBuilder_;
+      }
+      
+      // required uint64 size = 2;
+      private long size_ ;
+      public boolean hasSize() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getSize() {
+        return size_;
+      }
+      public Builder setSize(long value) {
+        bitField0_ |= 0x00000002;
+        size_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearSize() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        size_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:GetBlocksRequestProto)
+    }
+    
+    static {
+      defaultInstance = new GetBlocksRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetBlocksRequestProto)
+  }
+  
+  public interface GetBlocksResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .BlockWithLocationsProto blocks = 1;
+    boolean hasBlocks();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto getBlocks();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProtoOrBuilder getBlocksOrBuilder();
+  }
+  public static final class GetBlocksResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetBlocksResponseProtoOrBuilder {
+    // Use GetBlocksResponseProto.newBuilder() to construct.
+    private GetBlocksResponseProto(Builder builder) {
+      super(builder);
+    }
+    private GetBlocksResponseProto(boolean noInit) {}
+    
+    private static final GetBlocksResponseProto defaultInstance;
+    public static GetBlocksResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetBlocksResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlocksResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlocksResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .BlockWithLocationsProto blocks = 1;
+    public static final int BLOCKS_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto blocks_;
+    public boolean hasBlocks() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto getBlocks() {
+      return blocks_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProtoOrBuilder getBlocksOrBuilder() {
+      return blocks_;
+    }
+    
+    private void initFields() {
+      blocks_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlocks()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getBlocks().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, blocks_);
+      }
+      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, blocks_);
+      }
+      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.NamenodeProtocolProtos.GetBlocksResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlocks() == other.hasBlocks());
+      if (hasBlocks()) {
+        result = result && getBlocks()
+            .equals(other.getBlocks());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasBlocks()) {
+        hash = (37 * hash) + BLOCKS_FIELD_NUMBER;
+        hash = (53 * hash) + getBlocks().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto 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.NamenodeProtocolProtos.GetBlocksResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto 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.NamenodeProtocolProtos.GetBlocksResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto 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.NamenodeProtocolProtos.GetBlocksResponseProto 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.NamenodeProtocolProtos.GetBlocksResponseProto 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.NamenodeProtocolProtos.GetBlocksResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto 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.NamenodeProtocolProtos.GetBlocksResponseProto 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.NamenodeProtocolProtos.GetBlocksResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlocksResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlocksResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto.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_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.getDefaultInstance();
+        } else {
+          blocksBuilder_.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.NamenodeProtocolProtos.GetBlocksResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (blocksBuilder_ == null) {
+          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.NamenodeProtocolProtos.GetBlocksResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto.getDefaultInstance()) return this;
+        if (other.hasBlocks()) {
+          mergeBlocks(other.getBlocks());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBlocks()) {
+          
+          return false;
+        }
+        if (!getBlocks().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.BlockWithLocationsProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.newBuilder();
+              if (hasBlocks()) {
+                subBuilder.mergeFrom(getBlocks());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setBlocks(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .BlockWithLocationsProto blocks = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto blocks_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProtoOrBuilder> blocksBuilder_;
+      public boolean hasBlocks() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto getBlocks() {
+        if (blocksBuilder_ == null) {
+          return blocks_;
+        } else {
+          return blocksBuilder_.getMessage();
+        }
+      }
+      public Builder setBlocks(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto value) {
+        if (blocksBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          blocks_ = value;
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setBlocks(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.Builder builderForValue) {
+        if (blocksBuilder_ == null) {
+          blocks_ = builderForValue.build();
+          onChanged();
+        } else {
+          blocksBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeBlocks(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto value) {
+        if (blocksBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              blocks_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.getDefaultInstance()) {
+            blocks_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.newBuilder(blocks_).mergeFrom(value).buildPartial();
+          } else {
+            blocks_ = value;
+          }
+          onChanged();
+        } else {
+          blocksBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearBlocks() {
+        if (blocksBuilder_ == null) {
+          blocks_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.getDefaultInstance();
+          onChanged();
+        } else {
+          blocksBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.Builder getBlocksBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getBlocksFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProtoOrBuilder getBlocksOrBuilder() {
+        if (blocksBuilder_ != null) {
+          return blocksBuilder_.getMessageOrBuilder();
+        } else {
+          return blocks_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProtoOrBuilder> 
+          getBlocksFieldBuilder() {
+        if (blocksBuilder_ == null) {
+          blocksBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProtoOrBuilder>(
+                  blocks_,
+                  getParentForChildren(),
+                  isClean());
+          blocks_ = null;
+        }
+        return blocksBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:GetBlocksResponseProto)
+    }
+    
+    static {
+      defaultInstance = new GetBlocksResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetBlocksResponseProto)
+  }
+  
+  public interface GetBlockKeysRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class GetBlockKeysRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetBlockKeysRequestProtoOrBuilder {
+    // Use GetBlockKeysRequestProto.newBuilder() to construct.
+    private GetBlockKeysRequestProto(Builder builder) {
+      super(builder);
+    }
+    private GetBlockKeysRequestProto(boolean noInit) {}
+    
+    private static final GetBlockKeysRequestProto defaultInstance;
+    public static GetBlockKeysRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetBlockKeysRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlockKeysRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlockKeysRequestProto_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.NamenodeProtocolProtos.GetBlockKeysRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto) 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.NamenodeProtocolProtos.GetBlockKeysRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto 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.NamenodeProtocolProtos.GetBlockKeysRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto 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.NamenodeProtocolProtos.GetBlockKeysRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto 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.NamenodeProtocolProtos.GetBlockKeysRequestProto 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.NamenodeProtocolProtos.GetBlockKeysRequestProto 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.NamenodeProtocolProtos.GetBlockKeysRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto 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.NamenodeProtocolProtos.GetBlockKeysRequestProto 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.NamenodeProtocolProtos.GetBlockKeysRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlockKeysRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlockKeysRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto.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.NamenodeProtocolProtos.GetBlockKeysRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto.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:GetBlockKeysRequestProto)
+    }
+    
+    static {
+      defaultInstance = new GetBlockKeysRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetBlockKeysRequestProto)
+  }
+  
+  public interface GetBlockKeysResponseProtoOrBuilder
+      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 GetBlockKeysResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetBlockKeysResponseProtoOrBuilder {
+    // Use GetBlockKeysResponseProto.newBuilder() to construct.
+    private GetBlockKeysResponseProto(Builder builder) {
+      super(builder);
+    }
+    private GetBlockKeysResponseProto(boolean noInit) {}
+    
+    private static final GetBlockKeysResponseProto defaultInstance;
+    public static GetBlockKeysResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetBlockKeysResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlockKeysResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlockKeysResponseProto_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.NamenodeProtocolProtos.GetBlockKeysResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto) 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.NamenodeProtocolProtos.GetBlockKeysResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto 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.NamenodeProtocolProtos.GetBlockKeysResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto 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.NamenodeProtocolProtos.GetBlockKeysResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto 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.NamenodeProtocolProtos.GetBlockKeysResponseProto 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.NamenodeProtocolProtos.GetBlockKeysResponseProto 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.NamenodeProtocolProtos.GetBlockKeysResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto 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.NamenodeProtocolProtos.GetBlockKeysResponseProto 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.NamenodeProtocolProtos.GetBlockKeysResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlockKeysResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetBlockKeysResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.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.NamenodeProtocolProtos.GetBlockKeysResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto(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.NamenodeProtocolProtos.GetBlockKeysResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.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:GetBlockKeysResponseProto)
+    }
+    
+    static {
+      defaultInstance = new GetBlockKeysResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetBlockKeysResponseProto)
+  }
+  
+  public interface GetTransactionIdRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class GetTransactionIdRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetTransactionIdRequestProtoOrBuilder {
+    // Use GetTransactionIdRequestProto.newBuilder() to construct.
+    private GetTransactionIdRequestProto(Builder builder) {
+      super(builder);
+    }
+    private GetTransactionIdRequestProto(boolean noInit) {}
+    
+    private static final GetTransactionIdRequestProto defaultInstance;
+    public static GetTransactionIdRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetTransactionIdRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetTransactionIdRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetTransactionIdRequestProto_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.NamenodeProtocolProtos.GetTransactionIdRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto) 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.NamenodeProtocolProtos.GetTransactionIdRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto 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.NamenodeProtocolProtos.GetTransactionIdRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto 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.NamenodeProtocolProtos.GetTransactionIdRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto 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.NamenodeProtocolProtos.GetTransactionIdRequestProto 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.NamenodeProtocolProtos.GetTransactionIdRequestProto 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.NamenodeProtocolProtos.GetTransactionIdRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto 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.NamenodeProtocolProtos.GetTransactionIdRequestProto 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.NamenodeProtocolProtos.GetTransactionIdRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetTransactionIdRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetTransactionIdRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto.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.NamenodeProtocolProtos.GetTransactionIdRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto.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:GetTransactionIdRequestProto)
+    }
+    
+    static {
+      defaultInstance = new GetTransactionIdRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetTransactionIdRequestProto)
+  }
+  
+  public interface GetTransactionIdResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required uint64 txId = 1;
+    boolean hasTxId();
+    long getTxId();
+  }
+  public static final class GetTransactionIdResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetTransactionIdResponseProtoOrBuilder {
+    // Use GetTransactionIdResponseProto.newBuilder() to construct.
+    private GetTransactionIdResponseProto(Builder builder) {
+      super(builder);
+    }
+    private GetTransactionIdResponseProto(boolean noInit) {}
+    
+    private static final GetTransactionIdResponseProto defaultInstance;
+    public static GetTransactionIdResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetTransactionIdResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetTransactionIdResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetTransactionIdResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required uint64 txId = 1;
+    public static final int TXID_FIELD_NUMBER = 1;
+    private long txId_;
+    public boolean hasTxId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public long getTxId() {
+      return txId_;
+    }
+    
+    private void initFields() {
+      txId_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasTxId()) {
+        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, txId_);
+      }
+      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, txId_);
+      }
+      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.NamenodeProtocolProtos.GetTransactionIdResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasTxId() == other.hasTxId());
+      if (hasTxId()) {
+        result = result && (getTxId()
+            == other.getTxId());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTxId()) {
+        hash = (37 * hash) + TXID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTxId());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto 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.NamenodeProtocolProtos.GetTransactionIdResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto 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.NamenodeProtocolProtos.GetTransactionIdResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto 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.NamenodeProtocolProtos.GetTransactionIdResponseProto 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.NamenodeProtocolProtos.GetTransactionIdResponseProto 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.NamenodeProtocolProtos.GetTransactionIdResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto 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.NamenodeProtocolProtos.GetTransactionIdResponseProto 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.NamenodeProtocolProtos.GetTransactionIdResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetTransactionIdResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetTransactionIdResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.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();
+        txId_ = 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.NamenodeProtocolProtos.GetTransactionIdResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.txId_ = txId_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.getDefaultInstance()) return this;
+        if (other.hasTxId()) {
+          setTxId(other.getTxId());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasTxId()) {
+          
+          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;
+              txId_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required uint64 txId = 1;
+      private long txId_ ;
+      public boolean hasTxId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public long getTxId() {
+        return txId_;
+      }
+      public Builder setTxId(long value) {
+        bitField0_ |= 0x00000001;
+        txId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearTxId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        txId_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:GetTransactionIdResponseProto)
+    }
+    
+    static {
+      defaultInstance = new GetTransactionIdResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetTransactionIdResponseProto)
+  }
+  
+  public interface RollEditLogRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class RollEditLogRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements RollEditLogRequestProtoOrBuilder {
+    // Use RollEditLogRequestProto.newBuilder() to construct.
+    private RollEditLogRequestProto(Builder builder) {
+      super(builder);
+    }
+    private RollEditLogRequestProto(boolean noInit) {}
+    
+    private static final RollEditLogRequestProto defaultInstance;
+    public static RollEditLogRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public RollEditLogRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RollEditLogRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RollEditLogRequestProto_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.NamenodeProtocolProtos.RollEditLogRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto) 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.NamenodeProtocolProtos.RollEditLogRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto 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.NamenodeProtocolProtos.RollEditLogRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto 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.NamenodeProtocolProtos.RollEditLogRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto 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.NamenodeProtocolProtos.RollEditLogRequestProto 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.NamenodeProtocolProtos.RollEditLogRequestProto 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.NamenodeProtocolProtos.RollEditLogRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto 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.NamenodeProtocolProtos.RollEditLogRequestProto 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.NamenodeProtocolProtos.RollEditLogRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RollEditLogRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RollEditLogRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto.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.NamenodeProtocolProtos.RollEditLogRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto.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:RollEditLogRequestProto)
+    }
+    
+    static {
+      defaultInstance = new RollEditLogRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:RollEditLogRequestProto)
+  }
+  
+  public interface RollEditLogResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .CheckpointSignatureProto signature = 1;
+    boolean hasSignature();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto getSignature();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder getSignatureOrBuilder();
+  }
+  public static final class RollEditLogResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements RollEditLogResponseProtoOrBuilder {
+    // Use RollEditLogResponseProto.newBuilder() to construct.
+    private RollEditLogResponseProto(Builder builder) {
+      super(builder);
+    }
+    private RollEditLogResponseProto(boolean noInit) {}
+    
+    private static final RollEditLogResponseProto defaultInstance;
+    public static RollEditLogResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public RollEditLogResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RollEditLogResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RollEditLogResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .CheckpointSignatureProto signature = 1;
+    public static final int SIGNATURE_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto signature_;
+    public boolean hasSignature() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto getSignature() {
+      return signature_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder getSignatureOrBuilder() {
+      return signature_;
+    }
+    
+    private void initFields() {
+      signature_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasSignature()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getSignature().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, signature_);
+      }
+      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, signature_);
+      }
+      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.NamenodeProtocolProtos.RollEditLogResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasSignature() == other.hasSignature());
+      if (hasSignature()) {
+        result = result && getSignature()
+            .equals(other.getSignature());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasSignature()) {
+        hash = (37 * hash) + SIGNATURE_FIELD_NUMBER;
+        hash = (53 * hash) + getSignature().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto 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.NamenodeProtocolProtos.RollEditLogResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto 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.NamenodeProtocolProtos.RollEditLogResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto 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.NamenodeProtocolProtos.RollEditLogResponseProto 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.NamenodeProtocolProtos.RollEditLogResponseProto 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.NamenodeProtocolProtos.RollEditLogResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto 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.NamenodeProtocolProtos.RollEditLogResponseProto 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.NamenodeProtocolProtos.RollEditLogResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RollEditLogResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RollEditLogResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getSignatureFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (signatureBuilder_ == null) {
+          signature_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.getDefaultInstance();
+        } else {
+          signatureBuilder_.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.NamenodeProtocolProtos.RollEditLogResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (signatureBuilder_ == null) {
+          result.signature_ = signature_;
+        } else {
+          result.signature_ = signatureBuilder_.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.NamenodeProtocolProtos.RollEditLogResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.getDefaultInstance()) return this;
+        if (other.hasSignature()) {
+          mergeSignature(other.getSignature());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasSignature()) {
+          
+          return false;
+        }
+        if (!getSignature().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.CheckpointSignatureProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.newBuilder();
+              if (hasSignature()) {
+                subBuilder.mergeFrom(getSignature());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setSignature(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .CheckpointSignatureProto signature = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto signature_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder> signatureBuilder_;
+      public boolean hasSignature() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto getSignature() {
+        if (signatureBuilder_ == null) {
+          return signature_;
+        } else {
+          return signatureBuilder_.getMessage();
+        }
+      }
+      public Builder setSignature(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto value) {
+        if (signatureBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          signature_ = value;
+          onChanged();
+        } else {
+          signatureBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setSignature(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder builderForValue) {
+        if (signatureBuilder_ == null) {
+          signature_ = builderForValue.build();
+          onChanged();
+        } else {
+          signatureBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeSignature(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto value) {
+        if (signatureBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              signature_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.getDefaultInstance()) {
+            signature_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.newBuilder(signature_).mergeFrom(value).buildPartial();
+          } else {
+            signature_ = value;
+          }
+          onChanged();
+        } else {
+          signatureBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearSignature() {
+        if (signatureBuilder_ == null) {
+          signature_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.getDefaultInstance();
+          onChanged();
+        } else {
+          signatureBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder getSignatureBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getSignatureFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder getSignatureOrBuilder() {
+        if (signatureBuilder_ != null) {
+          return signatureBuilder_.getMessageOrBuilder();
+        } else {
+          return signature_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder> 
+          getSignatureFieldBuilder() {
+        if (signatureBuilder_ == null) {
+          signatureBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder>(
+                  signature_,
+                  getParentForChildren(),
+                  isClean());
+          signature_ = null;
+        }
+        return signatureBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:RollEditLogResponseProto)
+    }
+    
+    static {
+      defaultInstance = new RollEditLogResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:RollEditLogResponseProto)
+  }
+  
+  public interface ErrorReportRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .NamenodeRegistrationProto registartion = 1;
+    boolean hasRegistartion();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistartion();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder 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.NamenodeProtocolProtos.internal_static_ErrorReportRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_ErrorReportRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .NamenodeRegistrationProto registartion = 1;
+    public static final int REGISTARTION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registartion_;
+    public boolean hasRegistartion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistartion() {
+      return registartion_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder 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.HdfsProtos.NamenodeRegistrationProto.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.NamenodeProtocolProtos.ErrorReportRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_ErrorReportRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_ErrorReportRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.HdfsProtos.NamenodeRegistrationProto.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.NamenodeProtocolProtos.ErrorReportRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.HdfsProtos.NamenodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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 .NamenodeRegistrationProto registartion = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registartion_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> registartionBuilder_;
+      public boolean hasRegistartion() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistartion() {
+        if (registartionBuilder_ == null) {
+          return registartion_;
+        } else {
+          return registartionBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistartion(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto 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.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto value) {
+        if (registartionBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registartion_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance()) {
+            registartion_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registartionBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder getRegistartionBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistartionFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistartionOrBuilder() {
+        if (registartionBuilder_ != null) {
+          return registartionBuilder_.getMessageOrBuilder();
+        } else {
+          return registartion_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> 
+          getRegistartionFieldBuilder() {
+        if (registartionBuilder_ == null) {
+          registartionBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder>(
+                  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.NamenodeProtocolProtos.internal_static_ErrorReportResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_ErrorReportResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_ErrorReportResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.NamenodeProtocolProtos.ErrorReportResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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 RegisterRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .NamenodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+  }
+  public static final class RegisterRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements RegisterRequestProtoOrBuilder {
+    // Use RegisterRequestProto.newBuilder() to construct.
+    private RegisterRequestProto(Builder builder) {
+      super(builder);
+    }
+    private RegisterRequestProto(boolean noInit) {}
+    
+    private static final RegisterRequestProto defaultInstance;
+    public static RegisterRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public RegisterRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RegisterRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RegisterRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .NamenodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.NamenodeProtocolProtos.RegisterRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto) 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.NamenodeProtocolProtos.RegisterRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto 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.NamenodeProtocolProtos.RegisterRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto 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.NamenodeProtocolProtos.RegisterRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto 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.NamenodeProtocolProtos.RegisterRequestProto 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.NamenodeProtocolProtos.RegisterRequestProto 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.NamenodeProtocolProtos.RegisterRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto 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.NamenodeProtocolProtos.RegisterRequestProto 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.NamenodeProtocolProtos.RegisterRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RegisterRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RegisterRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto.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.HdfsProtos.NamenodeRegistrationProto.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.NamenodeProtocolProtos.RegisterRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto(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.NamenodeProtocolProtos.RegisterRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto.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.HdfsProtos.NamenodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .NamenodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto 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.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:RegisterRequestProto)
+    }
+    
+    static {
+      defaultInstance = new RegisterRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:RegisterRequestProto)
+  }
+  
+  public interface RegisterResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .NamenodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+  }
+  public static final class RegisterResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements RegisterResponseProtoOrBuilder {
+    // Use RegisterResponseProto.newBuilder() to construct.
+    private RegisterResponseProto(Builder builder) {
+      super(builder);
+    }
+    private RegisterResponseProto(boolean noInit) {}
+    
+    private static final RegisterResponseProto defaultInstance;
+    public static RegisterResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public RegisterResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RegisterResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RegisterResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .NamenodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.NamenodeProtocolProtos.RegisterResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto) 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.NamenodeProtocolProtos.RegisterResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto 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.NamenodeProtocolProtos.RegisterResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto 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.NamenodeProtocolProtos.RegisterResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto 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.NamenodeProtocolProtos.RegisterResponseProto 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.NamenodeProtocolProtos.RegisterResponseProto 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.NamenodeProtocolProtos.RegisterResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto 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.NamenodeProtocolProtos.RegisterResponseProto 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.NamenodeProtocolProtos.RegisterResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RegisterResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_RegisterResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.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.HdfsProtos.NamenodeRegistrationProto.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.NamenodeProtocolProtos.RegisterResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto(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.NamenodeProtocolProtos.RegisterResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.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.HdfsProtos.NamenodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .NamenodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto 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.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:RegisterResponseProto)
+    }
+    
+    static {
+      defaultInstance = new RegisterResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:RegisterResponseProto)
+  }
+  
+  public interface StartCheckpointRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .NamenodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+  }
+  public static final class StartCheckpointRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements StartCheckpointRequestProtoOrBuilder {
+    // Use StartCheckpointRequestProto.newBuilder() to construct.
+    private StartCheckpointRequestProto(Builder builder) {
+      super(builder);
+    }
+    private StartCheckpointRequestProto(boolean noInit) {}
+    
+    private static final StartCheckpointRequestProto defaultInstance;
+    public static StartCheckpointRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public StartCheckpointRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_StartCheckpointRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_StartCheckpointRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .NamenodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.NamenodeProtocolProtos.StartCheckpointRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto) 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.NamenodeProtocolProtos.StartCheckpointRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto 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.NamenodeProtocolProtos.StartCheckpointRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto 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.NamenodeProtocolProtos.StartCheckpointRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto 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.NamenodeProtocolProtos.StartCheckpointRequestProto 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.NamenodeProtocolProtos.StartCheckpointRequestProto 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.NamenodeProtocolProtos.StartCheckpointRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto 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.NamenodeProtocolProtos.StartCheckpointRequestProto 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.NamenodeProtocolProtos.StartCheckpointRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_StartCheckpointRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_StartCheckpointRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto.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.HdfsProtos.NamenodeRegistrationProto.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.NamenodeProtocolProtos.StartCheckpointRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto(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.NamenodeProtocolProtos.StartCheckpointRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto.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.HdfsProtos.NamenodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .NamenodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto 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.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:StartCheckpointRequestProto)
+    }
+    
+    static {
+      defaultInstance = new StartCheckpointRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:StartCheckpointRequestProto)
+  }
+  
+  public interface StartCheckpointResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .NamenodeCommandProto command = 1;
+    boolean hasCommand();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto getCommand();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProtoOrBuilder getCommandOrBuilder();
+  }
+  public static final class StartCheckpointResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements StartCheckpointResponseProtoOrBuilder {
+    // Use StartCheckpointResponseProto.newBuilder() to construct.
+    private StartCheckpointResponseProto(Builder builder) {
+      super(builder);
+    }
+    private StartCheckpointResponseProto(boolean noInit) {}
+    
+    private static final StartCheckpointResponseProto defaultInstance;
+    public static StartCheckpointResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public StartCheckpointResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_StartCheckpointResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_StartCheckpointResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .NamenodeCommandProto command = 1;
+    public static final int COMMAND_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto command_;
+    public boolean hasCommand() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto getCommand() {
+      return command_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProtoOrBuilder getCommandOrBuilder() {
+      return command_;
+    }
+    
+    private void initFields() {
+      command_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasCommand()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getCommand().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, command_);
+      }
+      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, command_);
+      }
+      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.NamenodeProtocolProtos.StartCheckpointResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasCommand() == other.hasCommand());
+      if (hasCommand()) {
+        result = result && getCommand()
+            .equals(other.getCommand());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasCommand()) {
+        hash = (37 * hash) + COMMAND_FIELD_NUMBER;
+        hash = (53 * hash) + getCommand().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto 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.NamenodeProtocolProtos.StartCheckpointResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto 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.NamenodeProtocolProtos.StartCheckpointResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto 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.NamenodeProtocolProtos.StartCheckpointResponseProto 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.NamenodeProtocolProtos.StartCheckpointResponseProto 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.NamenodeProtocolProtos.StartCheckpointResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto 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.NamenodeProtocolProtos.StartCheckpointResponseProto 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.NamenodeProtocolProtos.StartCheckpointResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_StartCheckpointResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_StartCheckpointResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getCommandFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (commandBuilder_ == null) {
+          command_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.getDefaultInstance();
+        } else {
+          commandBuilder_.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.NamenodeProtocolProtos.StartCheckpointResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (commandBuilder_ == null) {
+          result.command_ = command_;
+        } else {
+          result.command_ = commandBuilder_.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.NamenodeProtocolProtos.StartCheckpointResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.getDefaultInstance()) return this;
+        if (other.hasCommand()) {
+          mergeCommand(other.getCommand());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasCommand()) {
+          
+          return false;
+        }
+        if (!getCommand().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.NamenodeCommandProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.newBuilder();
+              if (hasCommand()) {
+                subBuilder.mergeFrom(getCommand());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setCommand(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .NamenodeCommandProto command = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto command_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProtoOrBuilder> commandBuilder_;
+      public boolean hasCommand() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto getCommand() {
+        if (commandBuilder_ == null) {
+          return command_;
+        } else {
+          return commandBuilder_.getMessage();
+        }
+      }
+      public Builder setCommand(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto value) {
+        if (commandBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          command_ = value;
+          onChanged();
+        } else {
+          commandBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setCommand(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.Builder builderForValue) {
+        if (commandBuilder_ == null) {
+          command_ = builderForValue.build();
+          onChanged();
+        } else {
+          commandBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeCommand(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto value) {
+        if (commandBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              command_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.getDefaultInstance()) {
+            command_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.newBuilder(command_).mergeFrom(value).buildPartial();
+          } else {
+            command_ = value;
+          }
+          onChanged();
+        } else {
+          commandBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearCommand() {
+        if (commandBuilder_ == null) {
+          command_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.getDefaultInstance();
+          onChanged();
+        } else {
+          commandBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.Builder getCommandBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getCommandFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProtoOrBuilder getCommandOrBuilder() {
+        if (commandBuilder_ != null) {
+          return commandBuilder_.getMessageOrBuilder();
+        } else {
+          return command_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProtoOrBuilder> 
+          getCommandFieldBuilder() {
+        if (commandBuilder_ == null) {
+          commandBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProtoOrBuilder>(
+                  command_,
+                  getParentForChildren(),
+                  isClean());
+          command_ = null;
+        }
+        return commandBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:StartCheckpointResponseProto)
+    }
+    
+    static {
+      defaultInstance = new StartCheckpointResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:StartCheckpointResponseProto)
+  }
+  
+  public interface EndCheckpointRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .NamenodeRegistrationProto registration = 1;
+    boolean hasRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder();
+    
+    // required .CheckpointSignatureProto signature = 2;
+    boolean hasSignature();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto getSignature();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder getSignatureOrBuilder();
+  }
+  public static final class EndCheckpointRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements EndCheckpointRequestProtoOrBuilder {
+    // Use EndCheckpointRequestProto.newBuilder() to construct.
+    private EndCheckpointRequestProto(Builder builder) {
+      super(builder);
+    }
+    private EndCheckpointRequestProto(boolean noInit) {}
+    
+    private static final EndCheckpointRequestProto defaultInstance;
+    public static EndCheckpointRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public EndCheckpointRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_EndCheckpointRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_EndCheckpointRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .NamenodeRegistrationProto registration = 1;
+    public static final int REGISTRATION_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_;
+    public boolean hasRegistration() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+      return registration_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+      return registration_;
+    }
+    
+    // required .CheckpointSignatureProto signature = 2;
+    public static final int SIGNATURE_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto signature_;
+    public boolean hasSignature() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto getSignature() {
+      return signature_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder getSignatureOrBuilder() {
+      return signature_;
+    }
+    
+    private void initFields() {
+      registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      signature_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.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 (!hasSignature()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegistration().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getSignature().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.writeMessage(2, signature_);
+      }
+      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
+          .computeMessageSize(2, signature_);
+      }
+      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.NamenodeProtocolProtos.EndCheckpointRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasRegistration() == other.hasRegistration());
+      if (hasRegistration()) {
+        result = result && getRegistration()
+            .equals(other.getRegistration());
+      }
+      result = result && (hasSignature() == other.hasSignature());
+      if (hasSignature()) {
+        result = result && getSignature()
+            .equals(other.getSignature());
+      }
+      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 (hasSignature()) {
+        hash = (37 * hash) + SIGNATURE_FIELD_NUMBER;
+        hash = (53 * hash) + getSignature().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto 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.NamenodeProtocolProtos.EndCheckpointRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto 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.NamenodeProtocolProtos.EndCheckpointRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto 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.NamenodeProtocolProtos.EndCheckpointRequestProto 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.NamenodeProtocolProtos.EndCheckpointRequestProto 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.NamenodeProtocolProtos.EndCheckpointRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto 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.NamenodeProtocolProtos.EndCheckpointRequestProto 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.NamenodeProtocolProtos.EndCheckpointRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_EndCheckpointRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_EndCheckpointRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getRegistrationFieldBuilder();
+          getSignatureFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (registrationBuilder_ == null) {
+          registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (signatureBuilder_ == null) {
+          signature_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.getDefaultInstance();
+        } else {
+          signatureBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto(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;
+        }
+        if (signatureBuilder_ == null) {
+          result.signature_ = signature_;
+        } else {
+          result.signature_ = signatureBuilder_.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.NamenodeProtocolProtos.EndCheckpointRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto.getDefaultInstance()) return this;
+        if (other.hasRegistration()) {
+          mergeRegistration(other.getRegistration());
+        }
+        if (other.hasSignature()) {
+          mergeSignature(other.getSignature());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasRegistration()) {
+          
+          return false;
+        }
+        if (!hasSignature()) {
+          
+          return false;
+        }
+        if (!getRegistration().isInitialized()) {
+          
+          return false;
+        }
+        if (!getSignature().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.NamenodeRegistrationProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.newBuilder();
+              if (hasRegistration()) {
+                subBuilder.mergeFrom(getRegistration());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setRegistration(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.newBuilder();
+              if (hasSignature()) {
+                subBuilder.mergeFrom(getSignature());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setSignature(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .NamenodeRegistrationProto registration = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto registration_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> registrationBuilder_;
+      public boolean hasRegistration() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto getRegistration() {
+        if (registrationBuilder_ == null) {
+          return registration_;
+        } else {
+          return registrationBuilder_.getMessage();
+        }
+      }
+      public Builder setRegistration(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto 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.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto value) {
+        if (registrationBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              registration_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance()) {
+            registration_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.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.HdfsProtos.NamenodeRegistrationProto.getDefaultInstance();
+          onChanged();
+        } else {
+          registrationBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder getRegistrationBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegistrationFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder getRegistrationOrBuilder() {
+        if (registrationBuilder_ != null) {
+          return registrationBuilder_.getMessageOrBuilder();
+        } else {
+          return registration_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder> 
+          getRegistrationFieldBuilder() {
+        if (registrationBuilder_ == null) {
+          registrationBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProtoOrBuilder>(
+                  registration_,
+                  getParentForChildren(),
+                  isClean());
+          registration_ = null;
+        }
+        return registrationBuilder_;
+      }
+      
+      // required .CheckpointSignatureProto signature = 2;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto signature_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder> signatureBuilder_;
+      public boolean hasSignature() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto getSignature() {
+        if (signatureBuilder_ == null) {
+          return signature_;
+        } else {
+          return signatureBuilder_.getMessage();
+        }
+      }
+      public Builder setSignature(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto value) {
+        if (signatureBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          signature_ = value;
+          onChanged();
+        } else {
+          signatureBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder setSignature(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder builderForValue) {
+        if (signatureBuilder_ == null) {
+          signature_ = builderForValue.build();
+          onChanged();
+        } else {
+          signatureBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder mergeSignature(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto value) {
+        if (signatureBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              signature_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.getDefaultInstance()) {
+            signature_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.newBuilder(signature_).mergeFrom(value).buildPartial();
+          } else {
+            signature_ = value;
+          }
+          onChanged();
+        } else {
+          signatureBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder clearSignature() {
+        if (signatureBuilder_ == null) {
+          signature_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.getDefaultInstance();
+          onChanged();
+        } else {
+          signatureBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder getSignatureBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getSignatureFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder getSignatureOrBuilder() {
+        if (signatureBuilder_ != null) {
+          return signatureBuilder_.getMessageOrBuilder();
+        } else {
+          return signature_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder> 
+          getSignatureFieldBuilder() {
+        if (signatureBuilder_ == null) {
+          signatureBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProtoOrBuilder>(
+                  signature_,
+                  getParentForChildren(),
+                  isClean());
+          signature_ = null;
+        }
+        return signatureBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:EndCheckpointRequestProto)
+    }
+    
+    static {
+      defaultInstance = new EndCheckpointRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:EndCheckpointRequestProto)
+  }
+  
+  public interface EndCheckpointResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class EndCheckpointResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements EndCheckpointResponseProtoOrBuilder {
+    // Use EndCheckpointResponseProto.newBuilder() to construct.
+    private EndCheckpointResponseProto(Builder builder) {
+      super(builder);
+    }
+    private EndCheckpointResponseProto(boolean noInit) {}
+    
+    private static final EndCheckpointResponseProto defaultInstance;
+    public static EndCheckpointResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public EndCheckpointResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_EndCheckpointResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_EndCheckpointResponseProto_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.NamenodeProtocolProtos.EndCheckpointResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto) 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.NamenodeProtocolProtos.EndCheckpointResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto 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.NamenodeProtocolProtos.EndCheckpointResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto 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.NamenodeProtocolProtos.EndCheckpointResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto 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.NamenodeProtocolProtos.EndCheckpointResponseProto 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.NamenodeProtocolProtos.EndCheckpointResponseProto 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.NamenodeProtocolProtos.EndCheckpointResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto 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.NamenodeProtocolProtos.EndCheckpointResponseProto 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.NamenodeProtocolProtos.EndCheckpointResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_EndCheckpointResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_EndCheckpointResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.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.NamenodeProtocolProtos.EndCheckpointResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.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:EndCheckpointResponseProto)
+    }
+    
+    static {
+      defaultInstance = new EndCheckpointResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:EndCheckpointResponseProto)
+  }
+  
+  public interface GetEditLogManifestRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required uint64 sinceTxId = 1;
+    boolean hasSinceTxId();
+    long getSinceTxId();
+  }
+  public static final class GetEditLogManifestRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetEditLogManifestRequestProtoOrBuilder {
+    // Use GetEditLogManifestRequestProto.newBuilder() to construct.
+    private GetEditLogManifestRequestProto(Builder builder) {
+      super(builder);
+    }
+    private GetEditLogManifestRequestProto(boolean noInit) {}
+    
+    private static final GetEditLogManifestRequestProto defaultInstance;
+    public static GetEditLogManifestRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetEditLogManifestRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetEditLogManifestRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetEditLogManifestRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required uint64 sinceTxId = 1;
+    public static final int SINCETXID_FIELD_NUMBER = 1;
+    private long sinceTxId_;
+    public boolean hasSinceTxId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public long getSinceTxId() {
+      return sinceTxId_;
+    }
+    
+    private void initFields() {
+      sinceTxId_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasSinceTxId()) {
+        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, sinceTxId_);
+      }
+      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, sinceTxId_);
+      }
+      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.NamenodeProtocolProtos.GetEditLogManifestRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasSinceTxId() == other.hasSinceTxId());
+      if (hasSinceTxId()) {
+        result = result && (getSinceTxId()
+            == other.getSinceTxId());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasSinceTxId()) {
+        hash = (37 * hash) + SINCETXID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getSinceTxId());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto 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.NamenodeProtocolProtos.GetEditLogManifestRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto 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.NamenodeProtocolProtos.GetEditLogManifestRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto 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.NamenodeProtocolProtos.GetEditLogManifestRequestProto 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.NamenodeProtocolProtos.GetEditLogManifestRequestProto 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.NamenodeProtocolProtos.GetEditLogManifestRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto 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.NamenodeProtocolProtos.GetEditLogManifestRequestProto 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.NamenodeProtocolProtos.GetEditLogManifestRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetEditLogManifestRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetEditLogManifestRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto.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();
+        sinceTxId_ = 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.NamenodeProtocolProtos.GetEditLogManifestRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.sinceTxId_ = sinceTxId_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto.getDefaultInstance()) return this;
+        if (other.hasSinceTxId()) {
+          setSinceTxId(other.getSinceTxId());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasSinceTxId()) {
+          
+          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;
+              sinceTxId_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required uint64 sinceTxId = 1;
+      private long sinceTxId_ ;
+      public boolean hasSinceTxId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public long getSinceTxId() {
+        return sinceTxId_;
+      }
+      public Builder setSinceTxId(long value) {
+        bitField0_ |= 0x00000001;
+        sinceTxId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearSinceTxId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        sinceTxId_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:GetEditLogManifestRequestProto)
+    }
+    
+    static {
+      defaultInstance = new GetEditLogManifestRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetEditLogManifestRequestProto)
+  }
+  
+  public interface GetEditLogManifestResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .RemoteEditLogManifestProto manifest = 1;
+    boolean hasManifest();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto getManifest();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProtoOrBuilder getManifestOrBuilder();
+  }
+  public static final class GetEditLogManifestResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetEditLogManifestResponseProtoOrBuilder {
+    // Use GetEditLogManifestResponseProto.newBuilder() to construct.
+    private GetEditLogManifestResponseProto(Builder builder) {
+      super(builder);
+    }
+    private GetEditLogManifestResponseProto(boolean noInit) {}
+    
+    private static final GetEditLogManifestResponseProto defaultInstance;
+    public static GetEditLogManifestResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetEditLogManifestResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetEditLogManifestResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetEditLogManifestResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .RemoteEditLogManifestProto manifest = 1;
+    public static final int MANIFEST_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto manifest_;
+    public boolean hasManifest() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto getManifest() {
+      return manifest_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProtoOrBuilder getManifestOrBuilder() {
+      return manifest_;
+    }
+    
+    private void initFields() {
+      manifest_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasManifest()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getManifest().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, manifest_);
+      }
+      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, manifest_);
+      }
+      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.NamenodeProtocolProtos.GetEditLogManifestResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasManifest() == other.hasManifest());
+      if (hasManifest()) {
+        result = result && getManifest()
+            .equals(other.getManifest());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasManifest()) {
+        hash = (37 * hash) + MANIFEST_FIELD_NUMBER;
+        hash = (53 * hash) + getManifest().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto 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.NamenodeProtocolProtos.GetEditLogManifestResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto 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.NamenodeProtocolProtos.GetEditLogManifestResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto 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.NamenodeProtocolProtos.GetEditLogManifestResponseProto 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.NamenodeProtocolProtos.GetEditLogManifestResponseProto 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.NamenodeProtocolProtos.GetEditLogManifestResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto 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.NamenodeProtocolProtos.GetEditLogManifestResponseProto 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.NamenodeProtocolProtos.GetEditLogManifestResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetEditLogManifestResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.internal_static_GetEditLogManifestResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getManifestFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (manifestBuilder_ == null) {
+          manifest_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.getDefaultInstance();
+        } else {
+          manifestBuilder_.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.NamenodeProtocolProtos.GetEditLogManifestResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (manifestBuilder_ == null) {
+          result.manifest_ = manifest_;
+        } else {
+          result.manifest_ = manifestBuilder_.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.NamenodeProtocolProtos.GetEditLogManifestResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.getDefaultInstance()) return this;
+        if (other.hasManifest()) {
+          mergeManifest(other.getManifest());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasManifest()) {
+          
+          return false;
+        }
+        if (!getManifest().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.RemoteEditLogManifestProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.newBuilder();
+              if (hasManifest()) {
+                subBuilder.mergeFrom(getManifest());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setManifest(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .RemoteEditLogManifestProto manifest = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto manifest_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProtoOrBuilder> manifestBuilder_;
+      public boolean hasManifest() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto getManifest() {
+        if (manifestBuilder_ == null) {
+          return manifest_;
+        } else {
+          return manifestBuilder_.getMessage();
+        }
+      }
+      public Builder setManifest(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto value) {
+        if (manifestBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          manifest_ = value;
+          onChanged();
+        } else {
+          manifestBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setManifest(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.Builder builderForValue) {
+        if (manifestBuilder_ == null) {
+          manifest_ = builderForValue.build();
+          onChanged();
+        } else {
+          manifestBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeManifest(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto value) {
+        if (manifestBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              manifest_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.getDefaultInstance()) {
+            manifest_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.newBuilder(manifest_).mergeFrom(value).buildPartial();
+          } else {
+            manifest_ = value;
+          }
+          onChanged();
+        } else {
+          manifestBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearManifest() {
+        if (manifestBuilder_ == null) {
+          manifest_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.getDefaultInstance();
+          onChanged();
+        } else {
+          manifestBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.Builder getManifestBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getManifestFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProtoOrBuilder getManifestOrBuilder() {
+        if (manifestBuilder_ != null) {
+          return manifestBuilder_.getMessageOrBuilder();
+        } else {
+          return manifest_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProtoOrBuilder> 
+          getManifestFieldBuilder() {
+        if (manifestBuilder_ == null) {
+          manifestBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProtoOrBuilder>(
+                  manifest_,
+                  getParentForChildren(),
+                  isClean());
+          manifest_ = null;
+        }
+        return manifestBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:GetEditLogManifestResponseProto)
+    }
+    
+    static {
+      defaultInstance = new GetEditLogManifestResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetEditLogManifestResponseProto)
+  }
+  
+  public static abstract class NamenodeProtocolService
+      implements com.google.protobuf.Service {
+    protected NamenodeProtocolService() {}
+    
+    public interface Interface {
+      public abstract void getBlocks(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto> done);
+      
+      public abstract void getBlockKeys(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto> done);
+      
+      public abstract void getTransationId(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto> done);
+      
+      public abstract void rollEditLog(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto> done);
+      
+      public abstract void errorReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto> done);
+      
+      public abstract void register(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto> done);
+      
+      public abstract void startCheckpoint(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto> done);
+      
+      public abstract void endCheckpoint(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto> done);
+      
+      public abstract void getEditLogManifest(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto> done);
+      
+    }
+    
+    public static com.google.protobuf.Service newReflectiveService(
+        final Interface impl) {
+      return new NamenodeProtocolService() {
+        @java.lang.Override
+        public  void getBlocks(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto> done) {
+          impl.getBlocks(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void getBlockKeys(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto> done) {
+          impl.getBlockKeys(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void getTransationId(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto> done) {
+          impl.getTransationId(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void rollEditLog(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto> done) {
+          impl.rollEditLog(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void errorReport(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto> done) {
+          impl.errorReport(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void register(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto> done) {
+          impl.register(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void startCheckpoint(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto> done) {
+          impl.startCheckpoint(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void endCheckpoint(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto> done) {
+          impl.endCheckpoint(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void getEditLogManifest(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto> done) {
+          impl.getEditLogManifest(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.getBlocks(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto)request);
+            case 1:
+              return impl.getBlockKeys(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto)request);
+            case 2:
+              return impl.getTransationId(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto)request);
+            case 3:
+              return impl.rollEditLog(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto)request);
+            case 4:
+              return impl.errorReport(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto)request);
+            case 5:
+              return impl.register(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto)request);
+            case 6:
+              return impl.startCheckpoint(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto)request);
+            case 7:
+              return impl.endCheckpoint(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto)request);
+            case 8:
+              return impl.getEditLogManifest(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto)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.NamenodeProtocolProtos.GetBlocksRequestProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto.getDefaultInstance();
+            case 2:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto.getDefaultInstance();
+            case 3:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto.getDefaultInstance();
+            case 4:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto.getDefaultInstance();
+            case 5:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto.getDefaultInstance();
+            case 6:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto.getDefaultInstance();
+            case 7:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto.getDefaultInstance();
+            case 8:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto.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.NamenodeProtocolProtos.GetBlocksResponseProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.getDefaultInstance();
+            case 2:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.getDefaultInstance();
+            case 3:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.getDefaultInstance();
+            case 4:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance();
+            case 5:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.getDefaultInstance();
+            case 6:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.getDefaultInstance();
+            case 7:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.getDefaultInstance();
+            case 8:
+              return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+      };
+    }
+    
+    public abstract void getBlocks(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto> done);
+    
+    public abstract void getBlockKeys(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto> done);
+    
+    public abstract void getTransationId(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto> done);
+    
+    public abstract void rollEditLog(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto> done);
+    
+    public abstract void errorReport(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto> done);
+    
+    public abstract void register(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto> done);
+    
+    public abstract void startCheckpoint(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto> done);
+    
+    public abstract void endCheckpoint(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto> done);
+    
+    public abstract void getEditLogManifest(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto> done);
+    
+    public static final
+        com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.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.getBlocks(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto>specializeCallback(
+              done));
+          return;
+        case 1:
+          this.getBlockKeys(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto>specializeCallback(
+              done));
+          return;
+        case 2:
+          this.getTransationId(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto>specializeCallback(
+              done));
+          return;
+        case 3:
+          this.rollEditLog(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto>specializeCallback(
+              done));
+          return;
+        case 4:
+          this.errorReport(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto>specializeCallback(
+              done));
+          return;
+        case 5:
+          this.register(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto>specializeCallback(
+              done));
+          return;
+        case 6:
+          this.startCheckpoint(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto>specializeCallback(
+              done));
+          return;
+        case 7:
+          this.endCheckpoint(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto>specializeCallback(
+              done));
+          return;
+        case 8:
+          this.getEditLogManifest(controller, (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto>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.NamenodeProtocolProtos.GetBlocksRequestProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto.getDefaultInstance();
+        case 2:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto.getDefaultInstance();
+        case 3:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto.getDefaultInstance();
+        case 4:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto.getDefaultInstance();
+        case 5:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto.getDefaultInstance();
+        case 6:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto.getDefaultInstance();
+        case 7:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto.getDefaultInstance();
+        case 8:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto.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.NamenodeProtocolProtos.GetBlocksResponseProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.getDefaultInstance();
+        case 2:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.getDefaultInstance();
+        case 3:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.getDefaultInstance();
+        case 4:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance();
+        case 5:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.getDefaultInstance();
+        case 6:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.getDefaultInstance();
+        case 7:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.getDefaultInstance();
+        case 8:
+          return org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.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.NamenodeProtocolProtos.NamenodeProtocolService 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 getBlocks(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto.getDefaultInstance()));
+      }
+      
+      public  void getBlockKeys(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.getDefaultInstance()));
+      }
+      
+      public  void getTransationId(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(2),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.getDefaultInstance()));
+      }
+      
+      public  void rollEditLog(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(3),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.getDefaultInstance()));
+      }
+      
+      public  void errorReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(4),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance()));
+      }
+      
+      public  void register(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(5),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.getDefaultInstance()));
+      }
+      
+      public  void startCheckpoint(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(6),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.getDefaultInstance()));
+      }
+      
+      public  void endCheckpoint(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(7),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.getDefaultInstance()));
+      }
+      
+      public  void getEditLogManifest(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(8),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.getDefaultInstance()));
+      }
+    }
+    
+    public static BlockingInterface newBlockingStub(
+        com.google.protobuf.BlockingRpcChannel channel) {
+      return new BlockingStub(channel);
+    }
+    
+    public interface BlockingInterface {
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto getBlocks(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto getBlockKeys(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto getTransationId(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto rollEditLog(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto errorReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto register(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto startCheckpoint(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto endCheckpoint(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto getEditLogManifest(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto 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.NamenodeProtocolProtos.GetBlocksResponseProto getBlocks(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto getBlockKeys(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto getTransationId(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(2),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto rollEditLog(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(3),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto errorReport(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(4),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto register(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(5),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto startCheckpoint(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(6),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto endCheckpoint(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(7),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto getEditLogManifest(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(8),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.getDefaultInstance());
+      }
+      
+    }
+  }
+  
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetBlocksRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetBlocksRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetBlocksResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetBlocksResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetBlockKeysRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetBlockKeysRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetBlockKeysResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetBlockKeysResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetTransactionIdRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetTransactionIdRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetTransactionIdResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetTransactionIdResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_RollEditLogRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_RollEditLogRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_RollEditLogResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_RollEditLogResponseProto_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_RegisterRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_RegisterRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_RegisterResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_RegisterResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_StartCheckpointRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_StartCheckpointRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_StartCheckpointResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_StartCheckpointResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_EndCheckpointRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_EndCheckpointRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_EndCheckpointResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_EndCheckpointResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetEditLogManifestRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetEditLogManifestRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetEditLogManifestResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetEditLogManifestResponseProto_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\026NamenodeProtocol.proto\032\nhdfs.proto\"I\n\025" +
+      "GetBlocksRequestProto\022\"\n\010datanode\030\001 \002(\0132" +
+      "\020.DatanodeIDProto\022\014\n\004size\030\002 \002(\004\"B\n\026GetBl" +
+      "ocksResponseProto\022(\n\006blocks\030\001 \002(\0132\030.Bloc" +
+      "kWithLocationsProto\"\032\n\030GetBlockKeysReque" +
+      "stProto\"B\n\031GetBlockKeysResponseProto\022%\n\004" +
+      "keys\030\001 \002(\0132\027.ExportedBlockKeysProto\"\036\n\034G" +
+      "etTransactionIdRequestProto\"-\n\035GetTransa" +
+      "ctionIdResponseProto\022\014\n\004txId\030\001 \002(\004\"\031\n\027Ro" +
+      "llEditLogRequestProto\"H\n\030RollEditLogResp",
+      "onseProto\022,\n\tsignature\030\001 \002(\0132\031.Checkpoin" +
+      "tSignatureProto\"k\n\027ErrorReportRequestPro" +
+      "to\0220\n\014registartion\030\001 \002(\0132\032.NamenodeRegis" +
+      "trationProto\022\021\n\terrorCode\030\002 \002(\r\022\013\n\003msg\030\003" +
+      " \002(\t\"\032\n\030ErrorReportResponseProto\"H\n\024Regi" +
+      "sterRequestProto\0220\n\014registration\030\001 \002(\0132\032" +
+      ".NamenodeRegistrationProto\"I\n\025RegisterRe" +
+      "sponseProto\0220\n\014registration\030\001 \002(\0132\032.Name" +
+      "nodeRegistrationProto\"O\n\033StartCheckpoint" +
+      "RequestProto\0220\n\014registration\030\001 \002(\0132\032.Nam",
+      "enodeRegistrationProto\"F\n\034StartCheckpoin" +
+      "tResponseProto\022&\n\007command\030\001 \002(\0132\025.Nameno" +
+      "deCommandProto\"{\n\031EndCheckpointRequestPr" +
+      "oto\0220\n\014registration\030\001 \002(\0132\032.NamenodeRegi" +
+      "strationProto\022,\n\tsignature\030\002 \002(\0132\031.Check" +
+      "pointSignatureProto\"\034\n\032EndCheckpointResp" +
+      "onseProto\"3\n\036GetEditLogManifestRequestPr" +
+      "oto\022\021\n\tsinceTxId\030\001 \002(\004\"P\n\037GetEditLogMani" +
+      "festResponseProto\022-\n\010manifest\030\001 \002(\0132\033.Re" +
+      "moteEditLogManifestProto2\246\005\n\027NamenodePro",
+      "tocolService\022<\n\tgetBlocks\022\026.GetBlocksReq" +
+      "uestProto\032\027.GetBlocksResponseProto\022E\n\014ge" +
+      "tBlockKeys\022\031.GetBlockKeysRequestProto\032\032." +
+      "GetBlockKeysResponseProto\022P\n\017getTransati" +
+      "onId\022\035.GetTransactionIdRequestProto\032\036.Ge" +
+      "tTransactionIdResponseProto\022B\n\013rollEditL" +
+      "og\022\030.RollEditLogRequestProto\032\031.RollEditL" +
+      "ogResponseProto\022B\n\013errorReport\022\030.ErrorRe" +
+      "portRequestProto\032\031.ErrorReportResponsePr" +
+      "oto\0229\n\010register\022\025.RegisterRequestProto\032\026",
+      ".RegisterResponseProto\022N\n\017startCheckpoin" +
+      "t\022\034.StartCheckpointRequestProto\032\035.StartC" +
+      "heckpointResponseProto\022H\n\rendCheckpoint\022" +
+      "\032.EndCheckpointRequestProto\032\033.EndCheckpo" +
+      "intResponseProto\022W\n\022getEditLogManifest\022\037" +
+      ".GetEditLogManifestRequestProto\032 .GetEdi" +
+      "tLogManifestResponseProtoBE\n%org.apache." +
+      "hadoop.hdfs.protocol.protoB\026NamenodeProt" +
+      "ocolProtos\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_GetBlocksRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_GetBlocksRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetBlocksRequestProto_descriptor,
+              new java.lang.String[] { "Datanode", "Size", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto.Builder.class);
+          internal_static_GetBlocksResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_GetBlocksResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetBlocksResponseProto_descriptor,
+              new java.lang.String[] { "Blocks", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto.Builder.class);
+          internal_static_GetBlockKeysRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(2);
+          internal_static_GetBlockKeysRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetBlockKeysRequestProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto.Builder.class);
+          internal_static_GetBlockKeysResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(3);
+          internal_static_GetBlockKeysResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetBlockKeysResponseProto_descriptor,
+              new java.lang.String[] { "Keys", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto.Builder.class);
+          internal_static_GetTransactionIdRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(4);
+          internal_static_GetTransactionIdRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetTransactionIdRequestProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto.Builder.class);
+          internal_static_GetTransactionIdResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(5);
+          internal_static_GetTransactionIdResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetTransactionIdResponseProto_descriptor,
+              new java.lang.String[] { "TxId", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto.Builder.class);
+          internal_static_RollEditLogRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(6);
+          internal_static_RollEditLogRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_RollEditLogRequestProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto.Builder.class);
+          internal_static_RollEditLogResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(7);
+          internal_static_RollEditLogResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_RollEditLogResponseProto_descriptor,
+              new java.lang.String[] { "Signature", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto.Builder.class);
+          internal_static_ErrorReportRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(8);
+          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.NamenodeProtocolProtos.ErrorReportRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto.Builder.class);
+          internal_static_ErrorReportResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(9);
+          internal_static_ErrorReportResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ErrorReportResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto.Builder.class);
+          internal_static_RegisterRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(10);
+          internal_static_RegisterRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_RegisterRequestProto_descriptor,
+              new java.lang.String[] { "Registration", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto.Builder.class);
+          internal_static_RegisterResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(11);
+          internal_static_RegisterResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_RegisterResponseProto_descriptor,
+              new java.lang.String[] { "Registration", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto.Builder.class);
+          internal_static_StartCheckpointRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(12);
+          internal_static_StartCheckpointRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_StartCheckpointRequestProto_descriptor,
+              new java.lang.String[] { "Registration", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto.Builder.class);
+          internal_static_StartCheckpointResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(13);
+          internal_static_StartCheckpointResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_StartCheckpointResponseProto_descriptor,
+              new java.lang.String[] { "Command", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto.Builder.class);
+          internal_static_EndCheckpointRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(14);
+          internal_static_EndCheckpointRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_EndCheckpointRequestProto_descriptor,
+              new java.lang.String[] { "Registration", "Signature", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto.Builder.class);
+          internal_static_EndCheckpointResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(15);
+          internal_static_EndCheckpointResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_EndCheckpointResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto.Builder.class);
+          internal_static_GetEditLogManifestRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(16);
+          internal_static_GetEditLogManifestRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetEditLogManifestRequestProto_descriptor,
+              new java.lang.String[] { "SinceTxId", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto.Builder.class);
+          internal_static_GetEditLogManifestResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(17);
+          internal_static_GetEditLogManifestResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetEditLogManifestResponseProto_descriptor,
+              new java.lang.String[] { "Manifest", },
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto.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)
+}

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolServerSideTranslatorR23.java

@@ -21,9 +21,13 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.token.Token;
 
 /**
  * This class is used on the server side.
@@ -116,4 +120,10 @@ public class ClientDatanodeProtocolServerSideTranslatorR23 implements
   public void deleteBlockPool(String bpid, boolean force) throws IOException {
     server.deleteBlockPool(bpid, force);
   }
+
+  @Override
+  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
+      Token<BlockTokenIdentifier> token) throws IOException {
+    return server.getBlockLocalPathInfo(block, token);
+  }
 }

+ 25 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolTranslatorR23.java

@@ -26,14 +26,17 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 
 
 /**
@@ -63,6 +66,23 @@ public class ClientDatanodeProtocolTranslatorR23 implements
     rpcProxy = createClientDatanodeProtocolProxy(addr, ticket, conf, factory);
   }
   
+  /**
+   * Constructor.
+   * @param datanodeid Datanode to connect to.
+   * @param conf Configuration.
+   * @param socketTimeout Socket timeout to use.
+   * @throws IOException
+   */
+  public ClientDatanodeProtocolTranslatorR23(DatanodeID datanodeid,
+      Configuration conf, int socketTimeout) throws IOException {
+    InetSocketAddress addr = NetUtils.createSocketAddr(datanodeid.getHost()
+        + ":" + datanodeid.getIpcPort());
+    rpcProxy = RPC.getProxy(ClientDatanodeWireProtocol.class,
+        ClientDatanodeWireProtocol.versionID, addr,
+        UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf), socketTimeout);
+  }
+
   static ClientDatanodeWireProtocol createClientDatanodeProtocolProxy(
       DatanodeID datanodeid, Configuration conf, int socketTimeout,
       LocatedBlock locatedBlock)
@@ -134,4 +154,9 @@ public class ClientDatanodeProtocolTranslatorR23 implements
 
   }
 
+  @Override
+  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
+      Token<BlockTokenIdentifier> token) throws IOException {
+    return rpcProxy.getBlockLocalPathInfo(block, token);
+  }
 }

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeWireProtocol.java

@@ -24,11 +24,15 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector;
 import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
 
 /** 
@@ -77,6 +81,13 @@ public interface ClientDatanodeWireProtocol extends VersionedProtocol {
    */
   void deleteBlockPool(String bpid, boolean force) throws IOException; 
   
+  /**
+   * The specification of this method matches that of
+   * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
+   */
+  BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
+      Token<BlockTokenIdentifier> token) throws IOException;
+  
   /**
    * This method is defined to get the protocol signature using 
    * the R23 protocol - hence we have added the suffix of 2 to the method name

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -39,7 +39,6 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -29,8 +29,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
-import java.util.TreeSet;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -20,14 +20,10 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.io.DataInput;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Queue;
-import java.util.Set;
-import java.util.TreeSet;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java

@@ -20,8 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java

@@ -20,9 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import java.util.NavigableSet;
-import java.util.TreeSet;
-
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;

+ 782 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

@@ -0,0 +1,782 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import static org.apache.hadoop.hdfs.server.common.Util.now;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.LinkedList;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
+import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
+import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
+import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * A thread per namenode to perform:
+ * <ul>
+ * <li> Pre-registration handshake with namenode</li>
+ * <li> Registration with namenode</li>
+ * <li> Send periodic heartbeats to the namenode</li>
+ * <li> Handle commands received from the namenode</li>
+ * </ul>
+ */
+@InterfaceAudience.Private
+class BPOfferService implements Runnable {
+  static final Log LOG = DataNode.LOG;
+  
+  final InetSocketAddress nnAddr;
+  
+  /**
+   * Information about the namespace that this service
+   * is registering with. This is assigned after
+   * the first phase of the handshake.
+   */
+  NamespaceInfo bpNSInfo;
+
+  /**
+   * The registration information for this block pool.
+   * This is assigned after the second phase of the
+   * handshake.
+   */
+  DatanodeRegistration bpRegistration;
+  
+  long lastBlockReport = 0;
+  long lastDeletedReport = 0;
+
+  boolean resetBlockReportTime = true;
+
+  Thread bpThread;
+  DatanodeProtocol bpNamenode;
+  private long lastHeartbeat = 0;
+  private volatile boolean initialized = false;
+  private final LinkedList<ReceivedDeletedBlockInfo> receivedAndDeletedBlockList 
+    = new LinkedList<ReceivedDeletedBlockInfo>();
+  private volatile int pendingReceivedRequests = 0;
+  private volatile boolean shouldServiceRun = true;
+  UpgradeManagerDatanode upgradeManager = null;
+  private final DataNode dn;
+  private final DNConf dnConf;
+
+  BPOfferService(InetSocketAddress nnAddr, DataNode dn) {
+    this.dn = dn;
+    this.nnAddr = nnAddr;
+    this.dnConf = dn.getDnConf();
+  }
+
+  /**
+   * returns true if BP thread has completed initialization of storage
+   * and has registered with the corresponding namenode
+   * @return true if initialized
+   */
+  public boolean isInitialized() {
+    return initialized;
+  }
+  
+  public boolean isAlive() {
+    return shouldServiceRun && bpThread.isAlive();
+  }
+  
+  public String getBlockPoolId() {
+    if (bpNSInfo != null) {
+      return bpNSInfo.getBlockPoolID();
+    } else {
+      LOG.warn("Block pool ID needed, but service not yet registered with NN",
+          new Exception("trace"));
+      return null;
+    }
+  }
+  
+  public NamespaceInfo getNamespaceInfo() {
+    return bpNSInfo;
+  }
+  
+  public String toString() {
+    if (bpNSInfo == null) {
+      // If we haven't yet connected to our NN, we don't yet know our
+      // own block pool ID.
+      // If _none_ of the block pools have connected yet, we don't even
+      // know the storage ID of this DN.
+      String storageId = dn.getStorageId();
+      if (storageId == null || "".equals(storageId)) {
+        storageId = "unknown";
+      }
+      return "Block pool <registering> (storage id " + storageId +
+        ") connecting to " + nnAddr;
+    } else {
+      return "Block pool " + getBlockPoolId() +
+        " (storage id " + dn.getStorageId() +
+        ") registered with " + nnAddr;
+    }
+  }
+  
+  InetSocketAddress getNNSocketAddress() {
+    return nnAddr;
+  }
+
+  /**
+   * Used to inject a spy NN in the unit tests.
+   */
+  @VisibleForTesting
+  void setNameNode(DatanodeProtocol dnProtocol) {
+    bpNamenode = dnProtocol;
+  }
+
+  /**
+   * Perform the first part of the handshake with the NameNode.
+   * This calls <code>versionRequest</code> to determine the NN's
+   * namespace and version info. It automatically retries until
+   * the NN responds or the DN is shutting down.
+   * 
+   * @return the NamespaceInfo
+   * @throws IncorrectVersionException if the remote NN does not match
+   * this DN's version
+   */
+  NamespaceInfo retrieveNamespaceInfo() throws IncorrectVersionException {
+    NamespaceInfo nsInfo = null;
+    while (shouldRun()) {
+      try {
+        nsInfo = bpNamenode.versionRequest();
+        LOG.debug(this + " received versionRequest response: " + nsInfo);
+        break;
+      } catch(SocketTimeoutException e) {  // namenode is busy
+        LOG.warn("Problem connecting to server: " + nnAddr);
+      } catch(IOException e ) {  // namenode is not available
+        LOG.warn("Problem connecting to server: " + nnAddr);
+      }
+      
+      // try again in a second
+      sleepAndLogInterrupts(5000, "requesting version info from NN");
+    }
+    
+    if (nsInfo != null) {
+      checkNNVersion(nsInfo);        
+    }
+    return nsInfo;
+  }
+
+  private void checkNNVersion(NamespaceInfo nsInfo)
+      throws IncorrectVersionException {
+    // build and layout versions should match
+    String nsBuildVer = nsInfo.getBuildVersion();
+    String stBuildVer = Storage.getBuildVersion();
+    if (!nsBuildVer.equals(stBuildVer)) {
+      LOG.warn("Data-node and name-node Build versions must be the same. " +
+        "Namenode build version: " + nsBuildVer + "Datanode " +
+        "build version: " + stBuildVer);
+      throw new IncorrectVersionException(nsBuildVer, "namenode", stBuildVer);
+    }
+
+    if (HdfsConstants.LAYOUT_VERSION != nsInfo.getLayoutVersion()) {
+      LOG.warn("Data-node and name-node layout versions must be the same." +
+        " Expected: "+ HdfsConstants.LAYOUT_VERSION +
+        " actual "+ bpNSInfo.getLayoutVersion());
+      throw new IncorrectVersionException(
+          bpNSInfo.getLayoutVersion(), "namenode");
+    }
+  }
+
+  private void connectToNNAndHandshake() throws IOException {
+    // get NN proxy
+    bpNamenode = (DatanodeProtocol)RPC.waitForProxy(DatanodeProtocol.class,
+          DatanodeProtocol.versionID, nnAddr, dn.getConf());
+
+    // First phase of the handshake with NN - get the namespace
+    // info.
+    bpNSInfo = retrieveNamespaceInfo();
+    
+    // Now that we know the namespace ID, etc, we can pass this to the DN.
+    // The DN can now initialize its local storage if we are the
+    // first BP to handshake, etc.
+    dn.initBlockPool(this);
+    
+    // Second phase of the handshake with the NN.
+    register();
+  }
+  
+  /**
+   * This methods  arranges for the data node to send the block report at 
+   * the next heartbeat.
+   */
+  void scheduleBlockReport(long delay) {
+    if (delay > 0) { // send BR after random delay
+      lastBlockReport = System.currentTimeMillis()
+      - ( dnConf.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
+    } else { // send at next heartbeat
+      lastBlockReport = lastHeartbeat - dnConf.blockReportInterval;
+    }
+    resetBlockReportTime = true; // reset future BRs for randomness
+  }
+
+  void reportBadBlocks(ExtendedBlock block) {
+    DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) };
+    LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) }; 
+    
+    try {
+      bpNamenode.reportBadBlocks(blocks);  
+    } catch (IOException e){
+      /* One common reason is that NameNode could be in safe mode.
+       * Should we keep on retrying in that case?
+       */
+      LOG.warn("Failed to report bad block " + block + " to namenode : "
+          + " Exception", e);
+    }
+    
+  }
+  
+  /**
+   * Report received blocks and delete hints to the Namenode
+   * 
+   * @throws IOException
+   */
+  private void reportReceivedDeletedBlocks() throws IOException {
+
+    // check if there are newly received blocks
+    ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
+    int currentReceivedRequestsCounter;
+    synchronized (receivedAndDeletedBlockList) {
+      currentReceivedRequestsCounter = pendingReceivedRequests;
+      int numBlocks = receivedAndDeletedBlockList.size();
+      if (numBlocks > 0) {
+        //
+        // Send newly-received and deleted blockids to namenode
+        //
+        receivedAndDeletedBlockArray = receivedAndDeletedBlockList
+            .toArray(new ReceivedDeletedBlockInfo[numBlocks]);
+      }
+    }
+    if (receivedAndDeletedBlockArray != null) {
+      bpNamenode.blockReceivedAndDeleted(bpRegistration, getBlockPoolId(),
+          receivedAndDeletedBlockArray);
+      synchronized (receivedAndDeletedBlockList) {
+        for (int i = 0; i < receivedAndDeletedBlockArray.length; i++) {
+          receivedAndDeletedBlockList.remove(receivedAndDeletedBlockArray[i]);
+        }
+        pendingReceivedRequests -= currentReceivedRequestsCounter;
+      }
+    }
+  }
+
+  /*
+   * Informing the name node could take a long long time! Should we wait
+   * till namenode is informed before responding with success to the
+   * client? For now we don't.
+   */
+  void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
+    if (block == null || delHint == null) {
+      throw new IllegalArgumentException(block == null ? "Block is null"
+          : "delHint is null");
+    }
+
+    if (!block.getBlockPoolId().equals(getBlockPoolId())) {
+      LOG.warn("BlockPool mismatch " + block.getBlockPoolId() + " vs. "
+          + getBlockPoolId());
+      return;
+    }
+
+    synchronized (receivedAndDeletedBlockList) {
+      receivedAndDeletedBlockList.add(new ReceivedDeletedBlockInfo(block
+          .getLocalBlock(), delHint));
+      pendingReceivedRequests++;
+      receivedAndDeletedBlockList.notifyAll();
+    }
+  }
+
+  void notifyNamenodeDeletedBlock(ExtendedBlock block) {
+    if (block == null) {
+      throw new IllegalArgumentException("Block is null");
+    }
+
+    if (!block.getBlockPoolId().equals(getBlockPoolId())) {
+      LOG.warn("BlockPool mismatch " + block.getBlockPoolId() + " vs. "
+          + getBlockPoolId());
+      return;
+    }
+
+    synchronized (receivedAndDeletedBlockList) {
+      receivedAndDeletedBlockList.add(new ReceivedDeletedBlockInfo(block
+          .getLocalBlock(), ReceivedDeletedBlockInfo.TODELETE_HINT));
+    }
+  }
+
+
+  /**
+   * Report the list blocks to the Namenode
+   * @throws IOException
+   */
+  DatanodeCommand blockReport() throws IOException {
+    // send block report if timer has expired.
+    DatanodeCommand cmd = null;
+    long startTime = now();
+    if (startTime - lastBlockReport > dnConf.blockReportInterval) {
+
+      // Create block report
+      long brCreateStartTime = now();
+      BlockListAsLongs bReport = dn.data.getBlockReport(getBlockPoolId());
+
+      // Send block report
+      long brSendStartTime = now();
+      cmd = bpNamenode.blockReport(bpRegistration, getBlockPoolId(), bReport
+          .getBlockListAsLongs());
+
+      // Log the block report processing stats from Datanode perspective
+      long brSendCost = now() - brSendStartTime;
+      long brCreateCost = brSendStartTime - brCreateStartTime;
+      dn.metrics.addBlockReport(brSendCost);
+      LOG.info("BlockReport of " + bReport.getNumberOfBlocks()
+          + " blocks took " + brCreateCost + " msec to generate and "
+          + brSendCost + " msecs for RPC and NN processing");
+
+      // If we have sent the first block report, then wait a random
+      // time before we start the periodic block reports.
+      if (resetBlockReportTime) {
+        lastBlockReport = startTime - DFSUtil.getRandom().nextInt((int)(dnConf.blockReportInterval));
+        resetBlockReportTime = false;
+      } else {
+        /* say the last block report was at 8:20:14. The current report
+         * should have started around 9:20:14 (default 1 hour interval).
+         * If current time is :
+         *   1) normal like 9:20:18, next report should be at 10:20:14
+         *   2) unexpected like 11:35:43, next report should be at 12:20:14
+         */
+        lastBlockReport += (now() - lastBlockReport) /
+        dnConf.blockReportInterval * dnConf.blockReportInterval;
+      }
+      LOG.info("sent block report, processed command:" + cmd);
+    }
+    return cmd;
+  }
+  
+  
+  DatanodeCommand [] sendHeartBeat() throws IOException {
+    return bpNamenode.sendHeartbeat(bpRegistration,
+        dn.data.getCapacity(),
+        dn.data.getDfsUsed(),
+        dn.data.getRemaining(),
+        dn.data.getBlockPoolUsed(getBlockPoolId()),
+        dn.xmitsInProgress.get(),
+        dn.getXceiverCount(), dn.data.getNumFailedVolumes());
+  }
+  
+  //This must be called only by blockPoolManager
+  void start() {
+    if ((bpThread != null) && (bpThread.isAlive())) {
+      //Thread is started already
+      return;
+    }
+    bpThread = new Thread(this, formatThreadName());
+    bpThread.setDaemon(true); // needed for JUnit testing
+    bpThread.start();
+  }
+  
+  private String formatThreadName() {
+    Collection<URI> dataDirs = DataNode.getStorageDirs(dn.getConf());
+    return "DataNode: [" +
+      StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "] " +
+      " heartbeating to " + nnAddr;
+  }
+  
+  //This must be called only by blockPoolManager.
+  void stop() {
+    shouldServiceRun = false;
+    if (bpThread != null) {
+        bpThread.interrupt();
+    }
+  }
+  
+  //This must be called only by blockPoolManager
+  void join() {
+    try {
+      if (bpThread != null) {
+        bpThread.join();
+      }
+    } catch (InterruptedException ie) { }
+  }
+  
+  //Cleanup method to be called by current thread before exiting.
+  private synchronized void cleanUp() {
+    
+    if(upgradeManager != null)
+      upgradeManager.shutdownUpgrade();
+    shouldServiceRun = false;
+    RPC.stopProxy(bpNamenode);
+    dn.shutdownBlockPool(this);
+  }
+
+  /**
+   * Main loop for each BP thread. Run until shutdown,
+   * forever calling remote NameNode functions.
+   */
+  private void offerService() throws Exception {
+    LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
+        + dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
+        + dnConf.blockReportInterval + "msec" + " Initial delay: "
+        + dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
+        + dnConf.heartBeatInterval);
+
+    //
+    // Now loop for a long time....
+    //
+    while (shouldRun()) {
+      try {
+        long startTime = now();
+
+        //
+        // Every so often, send heartbeat or block-report
+        //
+        if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
+          //
+          // All heartbeat messages include following info:
+          // -- Datanode name
+          // -- data transfer port
+          // -- Total capacity
+          // -- Bytes remaining
+          //
+          lastHeartbeat = startTime;
+          if (!dn.areHeartbeatsDisabledForTests()) {
+            DatanodeCommand[] cmds = sendHeartBeat();
+            dn.metrics.addHeartbeat(now() - startTime);
+
+            long startProcessCommands = now();
+            if (!processCommand(cmds))
+              continue;
+            long endProcessCommands = now();
+            if (endProcessCommands - startProcessCommands > 2000) {
+              LOG.info("Took " + (endProcessCommands - startProcessCommands) +
+                  "ms to process " + cmds.length + " commands from NN");
+            }
+          }
+        }
+        if (pendingReceivedRequests > 0
+            || (startTime - lastDeletedReport > dnConf.deleteReportInterval)) {
+          reportReceivedDeletedBlocks();
+          lastDeletedReport = startTime;
+        }
+
+        DatanodeCommand cmd = blockReport();
+        processCommand(cmd);
+
+        // Now safe to start scanning the block pool
+        if (dn.blockScanner != null) {
+          dn.blockScanner.addBlockPool(this.getBlockPoolId());
+        }
+
+        //
+        // There is no work to do;  sleep until hearbeat timer elapses, 
+        // or work arrives, and then iterate again.
+        //
+        long waitTime = dnConf.heartBeatInterval - 
+        (System.currentTimeMillis() - lastHeartbeat);
+        synchronized(receivedAndDeletedBlockList) {
+          if (waitTime > 0 && pendingReceivedRequests == 0) {
+            try {
+              receivedAndDeletedBlockList.wait(waitTime);
+            } catch (InterruptedException ie) {
+              LOG.warn("BPOfferService for " + this + " interrupted");
+            }
+          }
+        } // synchronized
+      } catch(RemoteException re) {
+        String reClass = re.getClassName();
+        if (UnregisteredNodeException.class.getName().equals(reClass) ||
+            DisallowedDatanodeException.class.getName().equals(reClass) ||
+            IncorrectVersionException.class.getName().equals(reClass)) {
+          LOG.warn(this + " is shutting down", re);
+          shouldServiceRun = false;
+          return;
+        }
+        LOG.warn("RemoteException in offerService", re);
+        try {
+          long sleepTime = Math.min(1000, dnConf.heartBeatInterval);
+          Thread.sleep(sleepTime);
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+        }
+      } catch (IOException e) {
+        LOG.warn("IOException in offerService", e);
+      }
+    } // while (shouldRun())
+  } // offerService
+
+  /**
+   * Register one bp with the corresponding NameNode
+   * <p>
+   * The bpDatanode needs to register with the namenode on startup in order
+   * 1) to report which storage it is serving now and 
+   * 2) to receive a registrationID
+   *  
+   * issued by the namenode to recognize registered datanodes.
+   * 
+   * @see FSNamesystem#registerDatanode(DatanodeRegistration)
+   * @throws IOException
+   */
+  void register() throws IOException {
+    Preconditions.checkState(bpNSInfo != null,
+        "register() should be called after handshake()");
+    
+    // The handshake() phase loaded the block pool storage
+    // off disk - so update the bpRegistration object from that info
+    bpRegistration = dn.createBPRegistration(bpNSInfo);
+
+    LOG.info(this + " beginning handshake with NN");
+
+    while (shouldRun()) {
+      try {
+        // Use returned registration from namenode with updated machine name.
+        bpRegistration = bpNamenode.registerDatanode(bpRegistration);
+        break;
+      } catch(SocketTimeoutException e) {  // namenode is busy
+        LOG.info("Problem connecting to server: " + nnAddr);
+        sleepAndLogInterrupts(1000, "connecting to server");
+      }
+    }
+    
+    LOG.info("Block pool " + this + " successfully registered with NN");
+    dn.bpRegistrationSucceeded(bpRegistration, getBlockPoolId());
+
+    // random short delay - helps scatter the BR from all DNs
+    scheduleBlockReport(dnConf.initialBlockReportDelay);
+  }
+
+
+  private void sleepAndLogInterrupts(int millis,
+      String stateString) {
+    try {
+      Thread.sleep(millis);
+    } catch (InterruptedException ie) {
+      LOG.info("BPOfferService " + this +
+          " interrupted while " + stateString);
+    }
+  }
+
+  /**
+   * No matter what kind of exception we get, keep retrying to offerService().
+   * That's the loop that connects to the NameNode and provides basic DataNode
+   * functionality.
+   *
+   * Only stop when "shouldRun" or "shouldServiceRun" is turned off, which can
+   * happen either at shutdown or due to refreshNamenodes.
+   */
+  @Override
+  public void run() {
+    LOG.info(this + " starting to offer service");
+
+    try {
+      // init stuff
+      try {
+        // setup storage
+        connectToNNAndHandshake();
+      } catch (IOException ioe) {
+        // Initial handshake, storage recovery or registration failed
+        // End BPOfferService thread
+        LOG.fatal("Initialization failed for block pool " + this, ioe);
+        return;
+      }
+
+      initialized = true; // bp is initialized;
+      
+      while (shouldRun()) {
+        try {
+          startDistributedUpgradeIfNeeded();
+          offerService();
+        } catch (Exception ex) {
+          LOG.error("Exception in BPOfferService for " + this, ex);
+          sleepAndLogInterrupts(5000, "offering service");
+        }
+      }
+    } catch (Throwable ex) {
+      LOG.warn("Unexpected exception in block pool " + this, ex);
+    } finally {
+      LOG.warn("Ending block pool service for: " + this);
+      cleanUp();
+    }
+  }
+
+  private boolean shouldRun() {
+    return shouldServiceRun && dn.shouldRun();
+  }
+
+  /**
+   * Process an array of datanode commands
+   * 
+   * @param cmds an array of datanode commands
+   * @return true if further processing may be required or false otherwise. 
+   */
+  private boolean processCommand(DatanodeCommand[] cmds) {
+    if (cmds != null) {
+      for (DatanodeCommand cmd : cmds) {
+        try {
+          if (processCommand(cmd) == false) {
+            return false;
+          }
+        } catch (IOException ioe) {
+          LOG.warn("Error processing datanode Command", ioe);
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * 
+   * @param cmd
+   * @return true if further processing may be required or false otherwise. 
+   * @throws IOException
+   */
+  private boolean processCommand(DatanodeCommand cmd) throws IOException {
+    if (cmd == null)
+      return true;
+    final BlockCommand bcmd = 
+      cmd instanceof BlockCommand? (BlockCommand)cmd: null;
+
+    switch(cmd.getAction()) {
+    case DatanodeProtocol.DNA_TRANSFER:
+      // Send a copy of a block to another datanode
+      dn.transferBlocks(bcmd.getBlockPoolId(), bcmd.getBlocks(), bcmd.getTargets());
+      dn.metrics.incrBlocksReplicated(bcmd.getBlocks().length);
+      break;
+    case DatanodeProtocol.DNA_INVALIDATE:
+      //
+      // Some local block(s) are obsolete and can be 
+      // safely garbage-collected.
+      //
+      Block toDelete[] = bcmd.getBlocks();
+      try {
+        if (dn.blockScanner != null) {
+          dn.blockScanner.deleteBlocks(bcmd.getBlockPoolId(), toDelete);
+        }
+        // using global fsdataset
+        dn.data.invalidate(bcmd.getBlockPoolId(), toDelete);
+      } catch(IOException e) {
+        dn.checkDiskError();
+        throw e;
+      }
+      dn.metrics.incrBlocksRemoved(toDelete.length);
+      break;
+    case DatanodeProtocol.DNA_SHUTDOWN:
+      // shut down the data node
+      shouldServiceRun = false;
+      return false;
+    case DatanodeProtocol.DNA_REGISTER:
+      // namenode requested a registration - at start or if NN lost contact
+      LOG.info("DatanodeCommand action: DNA_REGISTER");
+      if (shouldRun()) {
+        // re-retrieve namespace info to make sure that, if the NN
+        // was restarted, we still match its version (HDFS-2120)
+        retrieveNamespaceInfo();
+        // and re-register
+        register();
+      }
+      break;
+    case DatanodeProtocol.DNA_FINALIZE:
+      String bp = ((FinalizeCommand) cmd).getBlockPoolId(); 
+      assert getBlockPoolId().equals(bp) :
+        "BP " + getBlockPoolId() + " received DNA_FINALIZE " +
+        "for other block pool " + bp;
+
+      dn.finalizeUpgradeForPool(bp);
+      break;
+    case UpgradeCommand.UC_ACTION_START_UPGRADE:
+      // start distributed upgrade here
+      processDistributedUpgradeCommand((UpgradeCommand)cmd);
+      break;
+    case DatanodeProtocol.DNA_RECOVERBLOCK:
+      dn.recoverBlocks(((BlockRecoveryCommand)cmd).getRecoveringBlocks());
+      break;
+    case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
+      LOG.info("DatanodeCommand action: DNA_ACCESSKEYUPDATE");
+      if (dn.isBlockTokenEnabled) {
+        dn.blockPoolTokenSecretManager.setKeys(getBlockPoolId(), 
+            ((KeyUpdateCommand) cmd).getExportedKeys());
+      }
+      break;
+    case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
+      LOG.info("DatanodeCommand action: DNA_BALANCERBANDWIDTHUPDATE");
+      long bandwidth =
+                 ((BalancerBandwidthCommand) cmd).getBalancerBandwidthValue();
+      if (bandwidth > 0) {
+        DataXceiverServer dxcs =
+                     (DataXceiverServer) dn.dataXceiverServer.getRunnable();
+        dxcs.balanceThrottler.setBandwidth(bandwidth);
+      }
+      break;
+    default:
+      LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
+    }
+    return true;
+  }
+  
+  private void processDistributedUpgradeCommand(UpgradeCommand comm)
+  throws IOException {
+    UpgradeManagerDatanode upgradeManager = getUpgradeManager();
+    upgradeManager.processUpgradeCommand(comm);
+  }
+
+  synchronized UpgradeManagerDatanode getUpgradeManager() {
+    if(upgradeManager == null)
+      upgradeManager = 
+        new UpgradeManagerDatanode(dn, getBlockPoolId());
+    
+    return upgradeManager;
+  }
+  
+  /**
+   * Start distributed upgrade if it should be initiated by the data-node.
+   */
+  private void startDistributedUpgradeIfNeeded() throws IOException {
+    UpgradeManagerDatanode um = getUpgradeManager();
+    
+    if(!um.getUpgradeState())
+      return;
+    um.setUpgradeState(false, um.getUpgradeVersion());
+    um.startUpgrade();
+    return;
+  }
+
+}

+ 12 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java

@@ -26,9 +26,11 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 
-import org.apache.commons.httpclient.methods.GetMethod;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
 
 
 /**
@@ -36,7 +38,9 @@ import org.apache.hadoop.util.DataChecksum;
  * This is not related to the Block related functionality in Namenode.
  * The biggest part of data block metadata is CRC for the block.
  */
-class BlockMetadataHeader {
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockMetadataHeader {
 
   static final short METADATA_VERSION = FSDataset.METADATA_VERSION;
   
@@ -52,12 +56,14 @@ class BlockMetadataHeader {
     this.checksum = checksum;
     this.version = version;
   }
-    
-  short getVersion() {
+  
+  /** Get the version */
+  public short getVersion() {
     return version;
   }
 
-  DataChecksum getChecksum() {
+  /** Get the checksum */
+  public DataChecksum getChecksum() {
     return checksum;
   }
 
@@ -68,7 +74,7 @@ class BlockMetadataHeader {
    * @return Metadata Header
    * @throws IOException
    */
-  static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
+  public static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
     return readHeader(in.readShort(), in);
   }
   

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java

@@ -34,7 +34,6 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Random;
 import java.util.TreeSet;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -252,8 +251,9 @@ class BlockPoolSliceScanner {
      */
     long period = Math.min(scanPeriod, 
                            Math.max(blockMap.size(),1) * 600 * 1000L);
+    int periodInt = Math.abs((int)period);
     return System.currentTimeMillis() - scanPeriod + 
-        DFSUtil.getRandom().nextInt((int)period);    
+        DFSUtil.getRandom().nextInt(periodInt);
   }
 
   /** Adds block to list of blocks */

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -50,7 +50,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.PureJavaCrc32;
 
 /** A class that receives a block and writes to its own disk, meanwhile
  * may copies it to another site. If a throttler is provided,

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -31,7 +31,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.datanode.DataNode.BPOfferService;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 

文件差异内容过多而无法显示
+ 139 - 710
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java


+ 1 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -48,8 +48,6 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Receiver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.Builder;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProtoOrBuilder;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
@@ -128,7 +126,7 @@ class DataXceiver extends Receiver implements Runnable {
   public void run() {
     int opsProcessed = 0;
     Op op = null;
-    dataXceiverServer.childSockets.put(s, s);
+    dataXceiverServer.childSockets.add(s);
     try {
       int stdTimeout = s.getSoTimeout();
 
@@ -165,14 +163,6 @@ class DataXceiver extends Receiver implements Runnable {
           s.setSoTimeout(stdTimeout);
         }
 
-        // Make sure the xceiver count is not exceeded
-        int curXceiverCount = datanode.getXceiverCount();
-        if (curXceiverCount > dataXceiverServer.maxXceiverCount) {
-          throw new IOException("xceiverCount " + curXceiverCount
-                                + " exceeds the limit of concurrent xcievers "
-                                + dataXceiverServer.maxXceiverCount);
-        }
-
         opStartTime = now();
         processOp(op);
         ++opsProcessed;

+ 14 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -23,9 +23,9 @@ import java.net.Socket;
 import java.net.SocketTimeoutException;
 import java.nio.channels.AsynchronousCloseException;
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
-import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
@@ -48,8 +48,8 @@ class DataXceiverServer implements Runnable {
   ServerSocket ss;
   DataNode datanode;
   // Record all sockets opened for data transfer
-  Map<Socket, Socket> childSockets = Collections.synchronizedMap(
-                                       new HashMap<Socket, Socket>());
+  Set<Socket> childSockets = Collections.synchronizedSet(
+                                       new HashSet<Socket>());
   
   /**
    * Maximal number of concurrent xceivers per node.
@@ -135,6 +135,15 @@ class DataXceiverServer implements Runnable {
       try {
         s = ss.accept();
         s.setTcpNoDelay(true);
+
+        // Make sure the xceiver count is not exceeded
+        int curXceiverCount = datanode.getXceiverCount();
+        if (curXceiverCount > maxXceiverCount) {
+          throw new IOException("Xceiver count " + curXceiverCount
+              + " exceeds the limit of concurrent xcievers: "
+              + maxXceiverCount);
+        }
+
         new Daemon(datanode.threadGroup, new DataXceiver(s, datanode, this))
             .start();
       } catch (SocketTimeoutException ignored) {
@@ -184,7 +193,7 @@ class DataXceiverServer implements Runnable {
 
     // close all the sockets that were accepted earlier
     synchronized (childSockets) {
-      for (Iterator<Socket> it = childSockets.values().iterator();
+      for (Iterator<Socket> it = childSockets.iterator();
            it.hasNext();) {
         Socket thissock = it.next();
         try {

+ 18 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
@@ -459,7 +460,7 @@ public class FSDataset implements FSDatasetInterface {
         long metaFileLen = metaFile.length();
         int crcHeaderLen = DataChecksum.getChecksumHeaderSize();
         if (!blockFile.exists() || blockFileLen == 0 ||
-            !metaFile.exists() || metaFileLen < (long)crcHeaderLen) {
+            !metaFile.exists() || metaFileLen < crcHeaderLen) {
           return 0;
         }
         checksumIn = new DataInputStream(
@@ -578,7 +579,7 @@ public class FSDataset implements FSDatasetInterface {
      * reserved capacity.
      * @return the unreserved number of bytes left in this filesystem. May be zero.
      */
-    long getCapacity() throws IOException {
+    long getCapacity() {
       long remaining = usage.getCapacity() - reserved;
       return remaining > 0 ? remaining : 0;
     }
@@ -818,7 +819,7 @@ public class FSDataset implements FSDatasetInterface {
       return dfsUsed;
     }
 
-    private long getCapacity() throws IOException {
+    private long getCapacity() {
       long capacity = 0L;
       for (FSVolume vol : volumes) {
         capacity += vol.getCapacity();
@@ -1667,7 +1668,7 @@ public class FSDataset implements FSDatasetInterface {
     }
     if (!oldmeta.renameTo(newmeta)) {
       replicaInfo.setGenerationStamp(oldGS); // restore old GS
-      throw new IOException("Block " + (Block)replicaInfo + " reopen failed. " +
+      throw new IOException("Block " + replicaInfo + " reopen failed. " +
                             " Unable to move meta file  " + oldmeta +
                             " to " + newmeta);
     }
@@ -2018,7 +2019,7 @@ public class FSDataset implements FSDatasetInterface {
   /**
    * Find the file corresponding to the block and return it if it exists.
    */
-  File validateBlockFile(String bpid, Block b) throws IOException {
+  File validateBlockFile(String bpid, Block b) {
     //Should we check for metadata file too?
     File f = getFile(bpid, b);
     
@@ -2327,7 +2328,7 @@ public class FSDataset implements FSDatasetInterface {
         if (datanode.blockScanner != null) {
           datanode.blockScanner.addBlock(new ExtendedBlock(bpid, diskBlockInfo));
         }
-        DataNode.LOG.warn("Added missing block to memory " + (Block)diskBlockInfo);
+        DataNode.LOG.warn("Added missing block to memory " + diskBlockInfo);
         return;
       }
       /*
@@ -2600,7 +2601,7 @@ public class FSDataset implements FSDatasetInterface {
    * get list of all bpids
    * @return list of bpids
    */
-  public String [] getBPIdlist() throws IOException {
+  public String [] getBPIdlist() {
     return volumeMap.getBlockPoolList();
   }
   
@@ -2658,4 +2659,14 @@ public class FSDataset implements FSDatasetInterface {
       volume.deleteBPDirectories(bpid, force);
     }
   }
+  
+  @Override // FSDatasetInterface
+  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block)
+      throws IOException {
+    File datafile = getBlockFile(block);
+    File metafile = getMetaFile(datafile, block.getGenerationStamp());
+    BlockLocalPathInfo info = new BlockLocalPathInfo(block,
+        datafile.getAbsolutePath(), metafile.getAbsolutePath());
+    return info;
+  }
 }

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 
 import java.io.Closeable;
+import java.io.File;
 import java.io.FilterInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -31,6 +32,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
@@ -402,4 +404,9 @@ public interface FSDatasetInterface extends FSDatasetMBean {
    * @throws IOException
    */
   public void deleteBlockPool(String bpid, boolean force) throws IOException;
+  
+  /**
+   * Get {@link BlockLocalPathInfo} for the given block.
+   **/
+  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) throws IOException;
 }

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java

@@ -60,6 +60,7 @@ public class DataNodeMetrics {
   @Metric MutableCounterLong readsFromRemoteClient;
   @Metric MutableCounterLong writesFromLocalClient;
   @Metric MutableCounterLong writesFromRemoteClient;
+  @Metric MutableCounterLong blocksGetLocalPathInfo;
   
   @Metric MutableCounterLong volumeFailures;
 
@@ -165,4 +166,9 @@ public class DataNodeMetrics {
   public void incrVolumeFailures() {
     volumeFailures.incr();
   }
+
+  /** Increment for getBlockLocalPathInfo calls */
+  public void incrBlocksGetLocalPathInfo() {
+    blocksGetLocalPathInfo.incr();
+  }
 }

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java

@@ -17,14 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
-import java.util.zip.Checksum;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 
-import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 
 /**

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java

@@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.util.Daemon;
 

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java

@@ -23,8 +23,6 @@ import java.net.InetSocketAddress;
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java

@@ -26,10 +26,7 @@ import java.io.EOFException;
 import java.io.DataInputStream;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
 import com.google.common.annotations.VisibleForTesting;
 
 /**

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
-import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 
-import java.io.File;
 import java.io.FilterInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -34,7 +33,6 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream.LogHeaderCorruptException;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
@@ -57,8 +55,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.Holder;
-import org.apache.hadoop.io.IOUtils;
-
 import com.google.common.base.Joiner;
 
 public class FSEditLogLoader {

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java

@@ -19,8 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.List;
-
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;

+ 0 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java

@@ -23,28 +23,16 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.TreeSet;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
-import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
-import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-
-import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
 
 class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
   public static final Log LOG = LogFactory.getLog(

+ 21 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -305,7 +305,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException if loading fails
    */
   public static FSNamesystem loadFromDisk(Configuration conf) throws IOException {
-    FSImage fsImage = new FSImage(conf);
+    Collection<URI> namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
+    Collection<URI> namespaceEditsDirs = 
+      FSNamesystem.getNamespaceEditsDirs(conf);
+
+    if (namespaceDirs.size() == 1) {
+      LOG.warn("Only one " + DFS_NAMENODE_NAME_DIR_KEY
+          + " directory configured , beware data loss!");
+    }
+    if (namespaceEditsDirs.size() == 1) {
+      LOG.warn("Only one " + DFS_NAMENODE_EDITS_DIR_KEY
+          + " directory configured , beware data loss!");
+    }
+
+    FSImage fsImage = new FSImage(conf, namespaceDirs, namespaceEditsDirs);
     FSNamesystem namesystem = new FSNamesystem(conf, fsImage);
 
     long loadStart = now();
@@ -2060,10 +2073,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  /** Get the file info for a specific file.
+  /**
+   * Get the file info for a specific file.
+   *
    * @param src The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException 
-   *        if src refers to a symlinks
+   *        if src refers to a symlink
    *
    * @throws AccessControlException if access is denied
    * @throws UnresolvedLinkException if a symlink is encountered.
@@ -2271,6 +2286,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     // If the penultimate block is not COMPLETE, then it must be COMMITTED.
     if(nrCompleteBlocks < nrBlocks - 2 ||
        nrCompleteBlocks == nrBlocks - 2 &&
+         curBlock != null &&
          curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
       final String message = "DIR* NameSystem.internalReleaseLease: "
         + "attempt to release a create lock on "
@@ -2459,7 +2475,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
                 newtargets[i]);
           }
         }
-        if (closeFile) {
+        if ((closeFile) && (descriptors != null)) {
           // the file is getting closed. Insert block locations into blockManager.
           // Otherwise fsck will report these blocks as MISSING, especially if the
           // blocksReceived from Datanodes take a long time to arrive.
@@ -3283,6 +3299,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     /**
      * Checks consistency of the class state.
      * This is costly and currently called only in assert.
+     * @throws IOException 
      */
     private boolean isConsistent() {
       if (blockTotal == -1 && blockSafe == -1) {

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java

@@ -19,8 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.URL;
 
 import javax.net.SocketFactory;

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -372,14 +372,16 @@ public abstract class INode implements Comparable<byte[]>, FSInodeInfo {
 
   /**
    * Given some components, create a path name.
-   * @param components
+   * @param components The path components
+   * @param start index
+   * @param end index
    * @return concatenated path
    */
-  static String constructPath(byte[][] components, int start) {
+  static String constructPath(byte[][] components, int start, int end) {
     StringBuilder buf = new StringBuilder();
-    for (int i = start; i < components.length; i++) {
+    for (int i = start; i < end; i++) {
       buf.append(DFSUtil.bytes2String(components[i]));
-      if (i < components.length - 1) {
+      if (i < end - 1) {
         buf.append(Path.SEPARATOR);
       }
     }

+ 11 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -23,8 +23,6 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -191,18 +189,19 @@ class INodeDirectory extends INode {
         existing[index] = curNode;
       }
       if (curNode.isLink() && (!lastComp || (lastComp && resolveLink))) {
-        if(NameNode.stateChangeLog.isDebugEnabled()) {
+        final String path = constructPath(components, 0, components.length);
+        final String preceding = constructPath(components, 0, count);
+        final String remainder =
+          constructPath(components, count + 1, components.length);
+        final String link = DFSUtil.bytes2String(components[count]);
+        final String target = ((INodeSymlink)curNode).getLinkValue();
+        if (NameNode.stateChangeLog.isDebugEnabled()) {
           NameNode.stateChangeLog.debug("UnresolvedPathException " +
-              " count: " + count +
-              " componenent: " + DFSUtil.bytes2String(components[count]) +
-              " full path: " + constructPath(components, 0) +
-              " remaining path: " + constructPath(components, count+1) +
-              " symlink: " + ((INodeSymlink)curNode).getLinkValue());
+            " path: " + path + " preceding: " + preceding +
+            " count: " + count + " link: " + link + " target: " + target +
+            " remainder: " + remainder);
         }
-        final String linkTarget = ((INodeSymlink)curNode).getLinkValue();
-        throw new UnresolvedPathException(constructPath(components, 0),
-                                          constructPath(components, count+1),
-                                          linkTarget);
+        throw new UnresolvedPathException(path, preceding, remainder, target);
       }
       if (lastComp || !curNode.isDirectory()) {
         break;

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
@@ -145,7 +149,7 @@ public class NameNodeHttpServer {
             }
           };
 
-          boolean certSSL = conf.getBoolean("dfs.https.enable", false);
+          boolean certSSL = conf.getBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, false);
           boolean useKrb = UserGroupInformation.isSecurityEnabled();
           if (certSSL || useKrb) {
             boolean needClientAuth = conf.getBoolean(
@@ -156,14 +160,14 @@ public class NameNodeHttpServer {
                     DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT));
             Configuration sslConf = new HdfsConfiguration(false);
             if (certSSL) {
-              sslConf.addResource(conf.get(
-                  "dfs.https.server.keystore.resource", "ssl-server.xml"));
+              sslConf.addResource(conf.get(DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
+                  DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT));
             }
             httpServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth,
                 useKrb);
             // assume same ssl port for all datanodes
             InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf
-                .get("dfs.datanode.https.address", infoHost + ":" + 50475));
+                .get(DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 50475));
             httpServer.setAttribute("datanode.https.port", datanodeSslPort
                 .getPort());
           }

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -819,10 +819,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public String getLinkTarget(String path) throws IOException {
     nn.checkOperation(OperationCategory.READ);
     metrics.incrGetLinkTargetOps();
-    /* Resolves the first symlink in the given path, returning a
-     * new path consisting of the target of the symlink and any 
-     * remaining path components from the original path.
-     */
     try {
       HdfsFileStatus stat = namesystem.getFileInfo(path, false);
       if (stat != null) {

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -29,7 +29,6 @@ import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.TreeSet;
 
 import org.apache.commons.logging.Log;

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -380,7 +380,13 @@ class NamenodeJspHelper {
     final NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
     final Configuration conf = (Configuration) context
         .getAttribute(JspHelper.CURRENT_CONF);
-    final DatanodeID datanode = getRandomDatanode(nn);
+    // We can't redirect if there isn't a DN to redirect to.
+    // Lets instead show a proper error message.
+    if (nn.getNamesystem().getNumLiveDataNodes() < 1) {
+      throw new IOException("Can't browse the DFS since there are no " +
+          "live nodes available to redirect to.");
+    }
+    final DatanodeID datanode = getRandomDatanode(nn);;
     UserGroupInformation ugi = JspHelper.getUGI(context, request, conf);
     String tokenString = getDelegationToken(
         nn.getRpcServer(), request, conf, ugi);

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RenewDelegationTokenServlet.java

@@ -29,7 +29,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java

@@ -20,8 +20,6 @@ package org.apache.hadoop.hdfs.server.protocol;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.Comparator;
-
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.io.Writable;
 

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockCommandWritable.java

@@ -20,17 +20,12 @@ package org.apache.hadoop.hdfs.server.protocolR23Compatible;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.List;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocolR23Compatible.BlockWritable;
 import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeInfoWritable;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java

@@ -34,7 +34,6 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -40,7 +40,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoader.java

@@ -18,8 +18,6 @@
 package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
 import java.io.IOException;
-import java.util.Map;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitor.java

@@ -22,8 +22,6 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
-import org.apache.hadoop.hdfs.DeprecatedUTF8;
-
 import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.ByteToken;
 import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.ShortToken;
 import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.IntToken;

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitorFactory.java

@@ -18,8 +18,6 @@
 package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
 import java.io.IOException;
-import java.util.regex.Pattern;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java

@@ -18,8 +18,6 @@
 package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 
 import java.io.IOException;
-import java.util.regex.Pattern;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DirectBufferPool.java

@@ -19,11 +19,6 @@ package org.apache.hadoop.hdfs.util;
 
 import java.lang.ref.WeakReference;
 import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java

@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.NoSuchElementException;
 
@@ -615,4 +614,4 @@ public class LightWeightHashSet<T> implements Collection<T> {
   public boolean retainAll(Collection<?> c) {
     throw new UnsupportedOperationException("retainAll is not supported.");
   }
-}
+}

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -122,11 +122,17 @@ creations/deletions), or "all".</description>
 <property>
   <name>dfs.datanode.https.address</name>
   <value>0.0.0.0:50475</value>
+  <description>The datanode secure http server address and port.
+    If the port is 0 then the server will start on a free port.
+  </description>
 </property>
 
 <property>
   <name>dfs.namenode.https-address</name>
   <value>0.0.0.0:50470</value>
+  <description>The namenode secure http server address and port.
+    If the port is 0 then the server will start on a free port.
+  </description>
 </property>
 
  <property>
@@ -193,6 +199,14 @@ creations/deletions), or "all".</description>
       directories, for redundancy. </description>
 </property>
 
+<property>
+  <name>dfs.namenode.name.dir.restore</name>
+  <value>false</value>
+  <description>Set to true to enable NameNode to attempt recovering a
+      previously failed dfs.name.dir. When enabled, a recovery of any failed
+      directory is attempted during checkpoint.</description>
+</property>
+
 <property>
   <name>dfs.namenode.fs-limits.max-component-length</name>
   <value>0</value>

+ 78 - 0
hadoop-hdfs-project/hadoop-hdfs/src/proto/InterDatanodeProtocol.proto

@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "InterDatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Block with location information and new generation stamp
+ * to be used for recovery.
+ */
+message InitReplicaRecoveryRequestProto {
+  required RecoveringBlockProto block = 1;
+}
+
+/**
+ * Repica recovery information
+ */
+message InitReplicaRecoveryResponseProto {
+  required ReplicaState state = 1; // State fo the replica
+  required BlockProto block = 2;   // block information
+}
+
+/**
+ * Update replica with new generation stamp and length
+ */
+message UpdateReplicaUnderRecoveryRequestProto {
+  required ExtendedBlockProto block = 1; // Block identifier
+  required uint64 recoveryId = 2;        // New genstamp of the replica
+  required uint64 newLength = 3;         // New length of the replica
+}
+
+/**
+ * Response returns updated block information
+ */
+message UpdateReplicaUnderRecoveryResponseProto {
+  required ExtendedBlockProto block = 1; // Updated block information
+}
+
+/**
+ * Protocol used between datanodes for block recovery.
+ *
+ * See the request and response for details of rpc call.
+ */
+service InterDatanodeProtocolService {
+  /**
+   * Initialize recovery of a replica
+   */
+  rpc initReplicaRecovery(InitReplicaRecoveryRequestProto)
+      returns(InitReplicaRecoveryResponseProto);
+
+  /**
+   * Update a replica with new generation stamp and length
+   */
+  rpc updateReplicaUnderRecovery(UpdateReplicaUnderRecoveryRequestProto)
+      returns(UpdateReplicaUnderRecoveryResponseProto);
+}

+ 83 - 0
hadoop-hdfs-project/hadoop-hdfs/src/proto/JournalProtocol.proto

@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "JournalProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * registration - the registration info of the active NameNode
+ * firstTxnId - the first txid in the rolled edit log
+ * numTxns - Number of transactions in editlog
+ * records - bytes containing serialized journal records
+ */
+message JournalRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required uint64 firstTxnId = 2; // Transaction ID
+  required uint32 numTxns = 3;    // Transaction ID
+  required bytes records = 4;     // Journal record
+}
+
+/**
+ * void response
+ */
+message JournalResponseProto { 
+}
+
+/**
+ * registration - the registration info of the active NameNode
+ * txid - first txid in the new log
+ */
+message StartLogSegmentRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required uint64 txid = 2; // Transaction ID
+}
+
+/**
+ * void response
+ */
+message StartLogSegmentResponseProto { 
+}
+
+/**
+ * Protocol used to journal edits to a remote node. Currently,
+ * this is used to publish edits from the NameNode to a BackupNode.
+ *
+ * See the request and response for details of rpc call.
+ */
+service JournalProtocolService {
+  /**
+   * Request sent by active namenode to backup node via 
+   * EditLogBackupOutputStream to stream editlog records.
+   */
+  rpc journal(JournalRequestProto) returns (JournalResponseProto);
+
+  /**
+   * Request sent by active namenode to backup node to notify 
+   * that the NameNode has rolled its edit logs and is now writing a 
+   * new log segment.
+   */
+  rpc startLogSegment(StartLogSegmentRequestProto) 
+      returns (StartLogSegmentResponseProto);
+}

+ 225 - 0
hadoop-hdfs-project/hadoop-hdfs/src/proto/NamenodeProtocol.proto

@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "NamenodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Get list of blocks for a given datanode with the total length 
+ * of adding up to given size
+ * datanode - Datanode ID to get list of block from
+ * size - size to which the block lengths must add up to
+ */
+message GetBlocksRequestProto {
+  required DatanodeIDProto datanode = 1; // Datanode ID
+  required uint64 size = 2;              // Size in bytes
+}
+
+ 
+/**
+ * blocks - List of returned blocks
+ */
+message GetBlocksResponseProto {
+  required BlockWithLocationsProto blocks = 1; // List of blocks
+}
+
+/**
+ * void request
+ */
+message GetBlockKeysRequestProto {
+}
+
+/**
+ * keys - Information about block keys at the active namenode
+ */
+message GetBlockKeysResponseProto {
+  required ExportedBlockKeysProto keys = 1;
+}
+
+/**
+ * void request
+ */
+message GetTransactionIdRequestProto {
+}
+
+/**
+ * txId - Transaction ID of the most recently persisted edit log record
+ */
+message GetTransactionIdResponseProto {
+  required uint64 txId = 1;   // Transaction ID
+}
+
+/**
+ * void request
+ */
+message RollEditLogRequestProto {
+}
+
+/**
+ * signature - A unique token to identify checkpoint transaction
+ */
+message RollEditLogResponseProto {
+  required CheckpointSignatureProto signature = 1;
+}
+
+/**
+ * registartion - Namenode reporting the error
+ * errorCode - error code indicating the error
+ * msg - Free text description of the error
+ */
+message ErrorReportRequestProto {
+  required NamenodeRegistrationProto registartion = 1; // Registartion info
+  required uint32 errorCode = 2;  // Error code
+  required string msg = 3;        // Error message
+}
+
+/**
+ * void response
+ */
+message ErrorReportResponseProto {
+}
+
+/**
+ * registration - Information of the namenode registering with primary namenode
+ */
+message RegisterRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * registration - Updated registration information of the newly registered
+ *                datanode.
+ */
+message RegisterResponseProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * Start checkpoint request
+ * registration - Namenode that is starting the checkpoint
+ */
+message StartCheckpointRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * command - Command returned by the active namenode to be
+ *           be handled by the caller.
+ */
+message StartCheckpointResponseProto {
+  required NamenodeCommandProto command = 1;
+}
+
+/**
+ * End or finalize the previously started checkpoint
+ * registration - Namenode that is ending the checkpoint
+ * signature - unique token to identify checkpoint transaction,
+ *             that was received when checkpoint was started.
+ */
+message EndCheckpointRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required CheckpointSignatureProto signature = 2;
+}
+
+/**
+ * void response
+ */
+message EndCheckpointResponseProto {
+}
+
+/**
+ * sinceTxId - return the editlog information for transactions >= sinceTxId
+ */
+message GetEditLogManifestRequestProto {
+  required uint64 sinceTxId = 1;  // Transaction ID
+}
+
+/**
+ * manifest - Enumeration of editlogs from namenode for 
+ *            logs >= sinceTxId in the request
+ */
+message GetEditLogManifestResponseProto {
+  required RemoteEditLogManifestProto manifest = 1; 
+}
+
+/**
+ * Protocol used by the sub-ordinate namenode to send requests
+ * the active/primary namenode.
+ *
+ * See the request and response for details of rpc call.
+ */
+service NamenodeProtocolService {
+  /**
+   * Get list of blocks for a given datanode with length
+   * of blocks adding up to given size.
+   */
+  rpc getBlocks(GetBlocksRequestProto) returns(GetBlocksResponseProto);
+
+  /**
+   * Get the current block keys
+   */
+  rpc getBlockKeys(GetBlockKeysRequestProto) returns(GetBlockKeysResponseProto);
+
+  /**
+   * Get the transaction ID of the most recently persisted editlog record
+   */
+  rpc getTransationId(GetTransactionIdRequestProto) 
+      returns(GetTransactionIdResponseProto);
+
+  /**
+   * Close the current editlog and open a new one for checkpointing purposes
+   */
+  rpc rollEditLog(RollEditLogRequestProto) returns(RollEditLogResponseProto);
+
+  /**
+   * Report from a sub-ordinate namenode of an error to the active namenode.
+   * Active namenode may decide to unregister the reporting namenode 
+   * depending on the error.
+   */
+  rpc errorReport(ErrorReportRequestProto) returns(ErrorReportResponseProto);
+
+  /**
+   * Request to register a sub-ordinate namenode
+   */
+  rpc register(RegisterRequestProto) returns(RegisterResponseProto);
+
+  /**
+   * Request to start a checkpoint. 
+   */
+  rpc startCheckpoint(StartCheckpointRequestProto) 
+      returns(StartCheckpointResponseProto);
+
+  /**
+   * End of finalize the previously started checkpoint
+   */
+  rpc endCheckpoint(EndCheckpointRequestProto) 
+      returns(EndCheckpointResponseProto);
+
+  /**
+   * Get editlog manifests from the active namenode for all the editlogs
+   */
+  rpc getEditLogManifest(GetEditLogManifestRequestProto) 
+      returns(GetEditLogManifestResponseProto);
+}

+ 162 - 6
hadoop-hdfs-project/hadoop-hdfs/src/proto/hdfs.proto

@@ -53,6 +53,12 @@ message DatanodeIDProto {
   required uint32 ipcPort = 4;   // the port where the ipc Server is running
 }
 
+/**
+ * DatanodeID array
+ */
+message DatanodeIDsProto {
+  repeated DatanodeIDProto datanodes = 1;
+}
 
 /**
  * The status of a Datanode
@@ -76,7 +82,6 @@ message DatanodeInfoProto {
   optional AdminState adminState = 10;
 }
 
-
 /**
  * Summary of a file or directory
  */
@@ -152,10 +157,10 @@ message HdfsFileStatusProto {
   required string group = 6;
   required uint64 modification_time = 7;
   required uint64 access_time = 8;
-  //
+
   // Optional fields for symlink
-  optional bytes symlink = 9;         // if symlink, target encoded java UTF8 
-  //
+  optional bytes symlink = 9;             // if symlink, target encoded java UTF8 
+
   // Optional fields for file
   optional uint32 block_replication = 10; // Actually a short - only 16bits used
   optional uint64 blocksize = 11;
@@ -169,7 +174,7 @@ message FsServerDefaultsProto {
   required uint64 blockSize = 1;
   required uint32 bytesPerChecksum = 2;
   required uint32 writePacketSize = 3;
-  required uint32 replication = 4; // Actually a short - only 16bits used
+  required uint32 replication = 4; // Actually a short - only 16 bits used
   required uint32 fileBufferSize = 5;
 }
 
@@ -187,5 +192,156 @@ message DirectoryListingProto {
  */
 message UpgradeStatusReportProto {
   required uint32 version = 1;;
-  required uint32 upgradeStatus = 2; // Between 0 and 100 indicating the % complete
+  required uint32 upgradeStatus = 2; // % completed in range 0 & 100
+}
+
+/**
+ * Common node information shared by all the nodes in the cluster
+ */
+message StorageInfoProto {
+  required uint32 layoutVersion = 1; // Layout version of the file system
+  required uint32 namespceID = 2;    // File system namespace ID
+  required string clusterID = 3;     // ID of the cluster
+  required uint64 cTime = 4;         // File system creation time
+}
+
+/**
+ * Information sent by a namenode to identify itself to the primary namenode.
+ */
+message NamenodeRegistrationProto {
+  required string rpcAddress = 1;    // host:port of the namenode RPC address
+  required string httpAddress = 2;   // host:port of the namenode http server
+  enum NamenodeRoleProto {
+    NAMENODE = 1;
+    BACKUP = 2;
+    CHECKPOINT = 3;
+  }
+  required StorageInfoProto storageInfo = 3;  // Node information
+  optional NamenodeRoleProto role = 4;        // Namenode role
+}
+
+/**
+ * Unique signature to identify checkpoint transactions.
+ */
+message CheckpointSignatureProto {
+  required string blockPoolId = 1;
+  required uint64 mostRecentCheckpointTxId = 2;
+  required uint64 curSegmentTxId = 3;
+  required StorageInfoProto storageInfo = 4;
+}
+
+/**
+ * Command sent from one namenode to another namenode.
+ */
+message NamenodeCommandProto {
+  enum Type {
+    NamenodeCommand = 0;      // Base command
+    CheckPointCommand = 1;    // Check point command
+  }
+  required uint32 action = 1;
+  required Type type = 2;
+  optional CheckpointCommandProto checkpointCmd = 3;
+}
+
+/**
+ * Command returned from primary to checkpointing namenode.
+ * This command has checkpoint signature that identifies
+ * checkpoint transaction and is needed for further
+ * communication related to checkpointing.
+ */
+message CheckpointCommandProto {
+  // Unique signature to identify checkpoint transation
+  required CheckpointSignatureProto signature = 1; 
+
+  // If true, return transfer image to primary upon the completion of checkpoint
+  required bool needToReturnImage = 2;
+}
+
+/**
+ * Block information
+ */
+message BlockProto {
+  required uint64 blockId = 1;
+  required uint64 genStamp = 2;
+  optional uint64 numBytes = 3;
+}
+
+/**
+ * Block and datanodes where is it located
+ */
+message BlockWithLocationsProto {
+  required BlockProto block = 1;            // Block
+  repeated DatanodeIDProto datanodeIDs = 2; // Datanodes with replicas of the block
+}
+
+/**
+ * List of block with locations
+ */
+message BlocksWithLocationsProto {
+  repeated BlockWithLocationsProto blocks = 1;
+}
+
+/**
+ * Editlog information with available transactions
+ */
+message RemoteEditLogProto {
+  required uint64 startTxId = 1;  // Starting available edit log transaction
+  required uint64 endTxId = 2;    // Ending available edit log transaction
+}
+
+/**
+ * Enumeration of editlogs available on a remote namenode
+ */
+message RemoteEditLogManifestProto {
+  repeated RemoteEditLogProto logs = 1;
+}
+
+/**
+ * Namespace information that describes namespace on a namenode
+ */
+message NamespaceInfoProto {
+  required string buildVersion = 1;         // Software build version
+  required uint32 distUpgradeVersion = 2;   // Distributed upgrade version
+  required string blockPoolID = 3;          // block pool used by the namespace
+  required StorageInfoProto storageInfo = 4;// Noe information
 }
+
+/**
+ * Block access token information
+ */
+message BlockKeyProto {
+  required uint32 keyId = 1;      // Key identifier
+  required uint64 expiryDate = 2; // Expiry time in milliseconds
+  required bytes keyBytes = 3;    // Key secret
+}
+
+/**
+ * Current key and set of block keys at the namenode.
+ */
+message ExportedBlockKeysProto {
+  required bool isBlockTokenEnabled = 1;
+  required uint64 keyUpdateInterval = 2;
+  required uint64 tokenLifeTime = 3;
+  required BlockKeyProto currentKey = 4;
+  repeated BlockKeyProto allKeys = 5;
+}
+
+/**
+ * State of a block replica at a datanode
+ */
+enum ReplicaState {
+  FINALIZED = 0;  // State of a replica when it is not modified
+  RBW = 1;        // State of replica that is being written to
+  RWR = 2;        // State of replica that is waiting to be recovered
+  RUR = 3;        // State of replica that is under recovery
+  TEMPORARY = 4;  // State of replica that is created for replication
+}
+
+/**
+ * Block that needs to be recovered with at a given location
+ */
+message RecoveringBlockProto {
+  required uint64 newGenStamp = 1;      // New genstamp post recovery
+  required LocatedBlockProto block = 2; // Block to be recovered
+}
+

+ 0 - 41
hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj

@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.datanode;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * This aspect takes care about faults injected into datanode.DataXceiver 
- * class 
- */
-privileged public aspect DataXceiverAspects {
-  public static final Log LOG = LogFactory.getLog(DataXceiverAspects.class);
-
-  pointcut runXceiverThread(DataXceiver xceiver) :
-    execution (* run(..)) && target(xceiver);
-
-  void around (DataXceiver xceiver) : runXceiverThread(xceiver) {
-    if ("true".equals(System.getProperty("fi.enabledOOM"))) {
-      LOG.info("fi.enabledOOM is enabled");
-      throw new OutOfMemoryError("Pretend there's no more memory");
-    } else {
-    	proceed(xceiver);
-    }
-  }
-}

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.fi.DataTransferTestUtil.SleepAction;
 import org.apache.hadoop.fi.DataTransferTestUtil.VerificationAction;
 import org.apache.hadoop.fi.FiTestUtil;
 import org.apache.hadoop.fi.FiTestUtil.Action;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -56,8 +57,9 @@ public class TestFiDataTransferProtocol {
 
   static private FSDataOutputStream createFile(FileSystem fs, Path p
       ) throws IOException {
-    return fs.create(p, true, fs.getConf().getInt("io.file.buffer.size", 4096),
-        REPLICATION, BLOCKSIZE);
+    return fs.create(p, true,
+        fs.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY,
+            4096), REPLICATION, BLOCKSIZE);
   }
 
   {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.fi.DataTransferTestUtil.DataTransferTest;
 import org.apache.hadoop.fi.DataTransferTestUtil.SleepAction;
 import org.apache.hadoop.fi.DataTransferTestUtil.VerificationAction;
 import org.apache.hadoop.fi.FiTestUtil;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -65,8 +66,8 @@ public class TestFiDataTransferProtocol2 {
 
   static private FSDataOutputStream createFile(FileSystem fs, Path p
       ) throws IOException {
-    return fs.create(p, true, fs.getConf().getInt("io.file.buffer.size", 4096),
-        REPLICATION, BLOCKSIZE);
+    return fs.create(p, true, fs.getConf()
+        .getInt(IO_FILE_BUFFER_SIZE_KEY, 4096), REPLICATION, BLOCKSIZE);
   }
 
   {

+ 0 - 97
hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java

@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.datanode;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.util.concurrent.CountDownLatch;
-
-import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-/**
- * This is a test for DataXceiverServer when DataXceiver thread spawning is
- * failed due to OutOfMemoryError. Expected behavior is that DataXceiverServer
- * should not be exited. It should retry again after 30 seconds
- */
-public class TestFiDataXceiverServer {
-
-  @Test(timeout = 30000)
-  public void testOutOfMemoryErrorInDataXceiverServerRun() throws Exception {
-    final CountDownLatch latch = new CountDownLatch(1);
-    ServerSocket sock = new ServerSocket() {
-      @Override
-      public Socket accept() throws IOException {
-        return new Socket() {
-          @Override
-          public InetAddress getInetAddress() {
-            return super.getLocalAddress();
-          }
-
-          @Override
-          public SocketAddress getRemoteSocketAddress() {
-            return new InetSocketAddress(8080);
-          }
-
-          @Override
-          public SocketAddress getLocalSocketAddress() {
-            return new InetSocketAddress(0);
-          }
-
-          @Override
-          public synchronized void close() throws IOException {
-            latch.countDown();
-            super.close();
-          }
-          
-          @Override
-          public InputStream getInputStream() throws IOException {
-            return null;
-          }
-        };
-      }
-    };
-    Thread thread = null;
-    System.setProperty("fi.enabledOOM", "true");
-    DataNode dn = Mockito.mock(DataNode.class);
-    try {
-      Configuration conf = new Configuration();
-      Mockito.doReturn(conf).when(dn).getConf();
-      dn.shouldRun = true;
-      DataXceiverServer server = new DataXceiverServer(sock, conf, dn);
-      thread = new Thread(server);
-      thread.start();
-      latch.await();
-      assertTrue("Not running the thread", thread.isAlive());
-    } finally {
-      System.setProperty("fi.enabledOOM", "false");
-      dn.shouldRun = false;
-      if (null != thread)
-        thread.interrupt();
-      sock.close();
-    }
-  }
-}

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestHelperDFS.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.cli;
 
 import org.apache.hadoop.cli.util.CLICommandDFSAdmin;
-import org.apache.hadoop.cli.util.CLITestCmd;
 import org.xml.sax.SAXException;
 
 public class CLITestHelperDFS extends CLITestHelper {

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

@@ -29,7 +29,6 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.junit.After;
 import static org.junit.Assert.assertTrue;
 import org.junit.Before;
-import org.junit.Test;
 
 public class TestHDFSCLI extends CLITestHelperDFS {
 

+ 12 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSymlink.java

@@ -20,6 +20,9 @@ package org.apache.hadoop.fs;
 import java.io.*;
 import java.net.URI;
 
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.log4j.Level;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileContext;
@@ -28,9 +31,11 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import org.apache.hadoop.ipc.RemoteException;
+
 import static org.junit.Assert.*;
 import org.junit.Test;
 import org.junit.BeforeClass;
@@ -41,6 +46,10 @@ import org.junit.AfterClass;
  */
 public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
 
+  {
+    ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+  }
+
   private static MiniDFSCluster cluster;
   
   protected String getScheme() {
@@ -250,8 +259,8 @@ public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
     Path link = new Path(testBaseDir1(), "symlinkToFile");
     createAndWriteFile(file);
     fc.createSymlink(file, link, false);
-    FileStatus stat_file = fc.getFileStatus(file);
-    FileStatus stat_link = fc.getFileStatus(link);
-    assertEquals(stat_link.getOwner(), stat_file.getOwner());
+    FileStatus statFile = fc.getFileStatus(file);
+    FileStatus statLink = fc.getFileStatus(link);
+    assertEquals(statLink.getOwner(), statFile.getOwner());
   }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java

@@ -163,7 +163,7 @@ public class TestStickyBit extends TestCase {
     try {
       Configuration conf = new HdfsConfiguration();
       conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-      conf.setBoolean("dfs.support.append", true);
+      conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
 
       FileSystem hdfs = cluster.getFileSystem();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java

@@ -143,8 +143,8 @@ public class AppendTestUtil {
   public static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
       throws IOException {
     return fileSys.create(name, true,
-        fileSys.getConf().getInt("io.file.buffer.size", 4096),
-        (short) repl, (long) BLOCK_SIZE);
+        fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, BLOCK_SIZE);
   }
 
   /**

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.net.NetUtils;
@@ -148,7 +149,7 @@ public class BlockReaderTestUtil {
       sock, targetAddr.toString()+ ":" + block.getBlockId(), block,
       testBlock.getBlockToken(), 
       offset, lenToRead,
-      conf.getInt("io.file.buffer.size", 4096),
+      conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
       true, "");
   }
 

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -84,6 +84,7 @@ public class DFSTestUtil {
   
   private int maxLevels;// = 3;
   private int maxSize;// = 8*1024;
+  private int minSize = 1;
   private int nFiles;
   private MyFile[] files;
   
@@ -139,7 +140,7 @@ public class DFSTestUtil {
       long fidx = -1;
       while (fidx < 0) { fidx = gen.nextLong(); }
       name = name + Long.toString(fidx);
-      size = gen.nextInt(maxSize);
+      size = minSize + gen.nextInt(maxSize - minSize);
       seed = gen.nextLong();
     }
     

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.junit.AfterClass;
@@ -66,7 +67,7 @@ public class FileAppendTest4 {
   }
 
   @AfterClass
-  public static void tearDown() throws IOException {
+  public static void tearDown() {
     cluster.shutdown();
   }
   
@@ -91,7 +92,7 @@ public class FileAppendTest4 {
             new Path("foo"+ oldFileLen +"_"+ flushedBytes1 +"_"+ flushedBytes2);
           LOG.info("Creating file " + p);
           FSDataOutputStream out = fs.create(p, false, 
-              conf.getInt("io.file.buffer.size", 4096), 
+              conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), 
               REPLICATION, BLOCK_SIZE);
           out.write(contents, 0, oldFileLen);
           out.close();

+ 56 - 53
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -36,21 +36,22 @@ import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.math.stat.descriptive.rank.Min;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ClientDatanodeWireProtocol;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
@@ -323,8 +324,8 @@ public class MiniDFSCluster {
    * Servers will be started on free ports.
    * <p>
    * The caller must manage the creation of NameNode and DataNode directories
-   * and have already set {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
-   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} in the given conf.
+   * and have already set {@link #DFS_NAMENODE_NAME_DIR_KEY} and 
+   * {@link #DFS_DATANODE_DATA_DIR_KEY} in the given conf.
    * 
    * @param conf the base configuration to use in starting the servers.  This
    *          will be modified as necessary.
@@ -398,8 +399,8 @@ public class MiniDFSCluster {
    * @param format if true, format the NameNode and DataNodes before starting 
    *          up
    * @param manageDfsDirs if true, the data directories for servers will be
-   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
-   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          created and {@link #DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link #DFS_DATANODE_DATA_DIR_KEY} will be set in 
    *          the conf
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -430,8 +431,8 @@ public class MiniDFSCluster {
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param manageDfsDirs if true, the data directories for servers will be
-   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
-   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          created and {@link #DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link #DFS_DATANODE_DATA_DIR_KEY} will be set in 
    *          the conf
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -464,11 +465,11 @@ public class MiniDFSCluster {
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param manageNameDfsDirs if true, the data directories for servers will be
-   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
-   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          created and {@link #DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link #DFS_DATANODE_DATA_DIR_KEY} will be set in 
    *          the conf
    * @param manageDataDfsDirs if true, the data directories for datanodes will
-   *          be created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} 
+   *          be created and {@link #DFS_DATANODE_DATA_DIR_KEY} 
    *          set to same in the conf
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -513,6 +514,8 @@ public class MiniDFSCluster {
       try {
         Class<?> rpcEngine = conf.getClassByName(rpcEngineName);
         setRpcEngine(conf, NamenodeProtocols.class, rpcEngine);
+        setRpcEngine(conf, ClientNamenodeWireProtocol.class, rpcEngine);
+        setRpcEngine(conf, ClientDatanodeWireProtocol.class, rpcEngine);
         setRpcEngine(conf, NamenodeProtocol.class, rpcEngine);
         setRpcEngine(conf, ClientProtocol.class, rpcEngine);
         setRpcEngine(conf, DatanodeProtocol.class, rpcEngine);
@@ -524,15 +527,15 @@ public class MiniDFSCluster {
       }
 
       // disable service authorization, as it does not work with tunnelled RPC
-      conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
+      conf.setBoolean(HADOOP_SECURITY_AUTHORIZATION,
                       false);
     }
     
-    int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
-    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
-    conf.setClass(DFSConfigKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, 
+    int replication = conf.getInt(DFS_REPLICATION_KEY, 3);
+    conf.setInt(DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
+    conf.setInt(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
+    conf.setInt(DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
+    conf.setClass(NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, 
                    StaticMapping.class, DNSToSwitchMapping.class);
     
     Collection<String> nameserviceIds = DFSUtil.getNameServiceIds(conf);
@@ -540,8 +543,8 @@ public class MiniDFSCluster {
       federation = true;
   
     if (!federation) {
-      conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "127.0.0.1:" + nameNodePort);
-      conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:"
+      conf.set(FS_DEFAULT_NAME_KEY, "127.0.0.1:" + nameNodePort);
+      conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:"
           + nameNodeHttpPort);
       NameNode nn = createNameNode(0, conf, numDataNodes, manageNameDfsDirs,
           format, operation, clusterId);
@@ -585,7 +588,7 @@ public class MiniDFSCluster {
       initFederatedNamenodeAddress(conf, nameserviceId, nnPort);
       nnPort = nnPort == 0 ? 0 : nnPort + 2;
     }
-    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nameserviceIdList);
+    conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIdList);
   }
 
   /* For federated namenode initialize the address:port */
@@ -593,11 +596,11 @@ public class MiniDFSCluster {
       String nameserviceId, int nnPort) {
     // Set nameserviceId specific key
     String key = DFSUtil.addKeySuffixes(
-        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId);
+        DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId);
     conf.set(key, "127.0.0.1:0");
 
     key = DFSUtil.addKeySuffixes(
-        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId);
+        DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId);
     conf.set(key, "127.0.0.1:" + nnPort);
   }
   
@@ -618,10 +621,10 @@ public class MiniDFSCluster {
       StartupOption operation, String clusterId)
       throws IOException {
     if (manageNameDfsDirs) {
-      conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+      conf.set(DFS_NAMENODE_NAME_DIR_KEY,
           fileAsURI(new File(base_dir, "name" + (2*nnIndex + 1)))+","+
           fileAsURI(new File(base_dir, "name" + (2*nnIndex + 2))));
-      conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
+      conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,
           fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 1)))+","+
           fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 2))));
     }
@@ -646,17 +649,17 @@ public class MiniDFSCluster {
       int numDataNodes, boolean manageNameDfsDirs, boolean format,
       StartupOption operation, String clusterId, String nameserviceId)
       throws IOException {
-    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
+    conf.set(DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
     NameNode nn = createNameNode(nnIndex, conf, numDataNodes, manageNameDfsDirs,
         format, operation, clusterId);
     conf.set(DFSUtil.addKeySuffixes(
-        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId), NameNode
+        DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId), NameNode
         .getHostPortString(nn.getNameNodeAddress()));
     conf.set(DFSUtil.addKeySuffixes(
-        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId), NameNode
+        DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId), NameNode
         .getHostPortString(nn.getHttpAddress()));
     DFSUtil.setGenericConf(conf, nameserviceId, 
-        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+        DFS_NAMENODE_HTTP_ADDRESS_KEY);
     nameNodes[nnIndex] = new NameNodeInfo(nn, new Configuration(conf));
   }
 
@@ -736,7 +739,7 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set 
+   *          created and {@link #DFS_DATANODE_DATA_DIR_KEY} will be set 
    *          in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -768,7 +771,7 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          created and {@link #DFS_DATANODE_DATA_DIR_KEY} will be 
    *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -802,7 +805,7 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          created and {@link #DFS_DATANODE_DATA_DIR_KEY} will be 
    *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -820,12 +823,12 @@ public class MiniDFSCluster {
                              long[] simulatedCapacities,
                              boolean setupHostsFile,
                              boolean checkDataNodeAddrConfig) throws IOException {
-    conf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
+    conf.set(DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
 
     int curDatanodesNum = dataNodes.size();
     // for mincluster's the default initialDelay for BRs is 0
-    if (conf.get(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
-      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, 0);
+    if (conf.get(DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
+      conf.setLong(DFS_BLOCKREPORT_INITIAL_DELAY_KEY, 0);
     }
     // If minicluster's name node is null assume that the conf has been
     // set with the right address:port of the name node.
@@ -872,8 +875,8 @@ public class MiniDFSCluster {
                                 + i + ": " + dir1 + " or " + dir2);
         }
         String dirs = fileAsURI(dir1) + "," + fileAsURI(dir2);
-        dnConf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs);
-        conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs);
+        dnConf.set(DFS_DATANODE_DATA_DIR_KEY, dirs);
+        conf.set(DFS_DATANODE_DATA_DIR_KEY, dirs);
       }
       if (simulatedCapacities != null) {
         dnConf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
@@ -902,7 +905,7 @@ public class MiniDFSCluster {
       DataNode dn = DataNode.instantiateDataNode(dnArgs, dnConf);
       if(dn == null)
         throw new IOException("Cannot start DataNode in "
-            + dnConf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
+            + dnConf.get(DFS_DATANODE_DATA_DIR_KEY));
       //since the HDFS does things based on IP:port, we need to add the mapping
       //for IP:port to rackId
       String ipAddr = dn.getSelfAddr().getAddress().getHostAddress();
@@ -1318,7 +1321,7 @@ public class MiniDFSCluster {
     Configuration newconf = new HdfsConfiguration(conf); // save cloned config
     if (keepPort) {
       InetSocketAddress addr = dnprop.datanode.getSelfAddr();
-      conf.set("dfs.datanode.address", addr.getAddress().getHostAddress() + ":"
+      conf.set(DFS_DATANODE_ADDRESS_KEY, addr.getAddress().getHostAddress() + ":"
           + addr.getPort());
     }
     dataNodes.add(new DataNodeProperties(DataNode.createDataNode(args, conf),
@@ -1445,10 +1448,10 @@ public class MiniDFSCluster {
   /**
    * @return a http URL
    */
-  public String getHttpUri(int nnIndex) throws IOException {
+  public String getHttpUri(int nnIndex) {
     return "http://"
         + nameNodes[nnIndex].conf
-            .get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+            .get(DFS_NAMENODE_HTTP_ADDRESS_KEY);
   }
   
   /**
@@ -1457,7 +1460,7 @@ public class MiniDFSCluster {
   public HftpFileSystem getHftpFileSystem(int nnIndex) throws IOException {
     String uri = "hftp://"
         + nameNodes[nnIndex].conf
-            .get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+            .get(DFS_NAMENODE_HTTP_ADDRESS_KEY);
     try {
       return (HftpFileSystem)FileSystem.get(new URI(uri), conf);
     } catch (URISyntaxException e) {
@@ -1907,9 +1910,9 @@ public class MiniDFSCluster {
     nameNodes = newlist;
     String nameserviceId = NAMESERVICE_ID_PREFIX + (nnIndex + 1);
     
-    String nameserviceIds = conf.get(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES);
+    String nameserviceIds = conf.get(DFS_FEDERATION_NAMESERVICES);
     nameserviceIds += "," + nameserviceId;
-    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nameserviceIds);
+    conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIds);
     
     initFederatedNamenodeAddress(conf, nameserviceId, namenodePort);
     createFederatedNameNode(nnIndex, conf, numDataNodes, true, true, null,
@@ -1942,28 +1945,28 @@ public class MiniDFSCluster {
   private void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,
                            boolean checkDataNodeAddrConfig) throws IOException {
     if (setupHostsFile) {
-      String hostsFile = conf.get(DFSConfigKeys.DFS_HOSTS, "").trim();
+      String hostsFile = conf.get(DFS_HOSTS, "").trim();
       if (hostsFile.length() == 0) {
         throw new IOException("Parameter dfs.hosts is not setup in conf");
       }
       // Setup datanode in the include file, if it is defined in the conf
       String address = "127.0.0.1:" + getFreeSocketPort();
       if (checkDataNodeAddrConfig) {
-        conf.setIfUnset("dfs.datanode.address", address);
+        conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, address);
       } else {
-        conf.set("dfs.datanode.address", address);
+        conf.set(DFS_DATANODE_ADDRESS_KEY, address);
       }
       addToFile(hostsFile, address);
       LOG.info("Adding datanode " + address + " to hosts file " + hostsFile);
     } else {
       if (checkDataNodeAddrConfig) {
-        conf.setIfUnset("dfs.datanode.address", "127.0.0.1:0");
-        conf.setIfUnset("dfs.datanode.http.address", "127.0.0.1:0");
-        conf.setIfUnset("dfs.datanode.ipc.address", "127.0.0.1:0");
+        conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
+        conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+        conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
       } else {
-        conf.set("dfs.datanode.address", "127.0.0.1:0");
-        conf.set("dfs.datanode.http.address", "127.0.0.1:0");
-        conf.set("dfs.datanode.ipc.address", "127.0.0.1:0");
+        conf.set(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
+        conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+        conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
       }
     }
   }

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java

@@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java

@@ -24,6 +24,7 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -80,9 +81,9 @@ public class TestBlockMissingException extends TestCase {
   //
   private void createOldFile(FileSystem fileSys, Path name, int repl, int numBlocks, long blocksize)
     throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                                            (short)repl, blocksize);
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short) repl, blocksize);
     // fill data into file
     final byte[] b = new byte[(int)blocksize];
     for (int i = 0; i < numBlocks; i++) {

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java

@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.io.IOUtils;
@@ -41,7 +40,7 @@ public class TestClientProtocolForPipelineRecovery {
   @Test public void testGetNewStamp() throws IOException {
     int numDataNodes = 1;
     Configuration conf = new HdfsConfiguration();
-    conf.setBoolean("dfs.support.append", true);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     try {
       cluster.waitActive();

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java

@@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -80,7 +81,7 @@ public class TestClientReportBadBlock {
         .build();
     cluster.waitActive();
     dfs = (DistributedFileSystem) cluster.getFileSystem();
-    buffersize = conf.getInt("io.file.buffer.size", 4096);
+    buffersize = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
   }
 
   @After

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java

@@ -20,8 +20,6 @@ package org.apache.hadoop.hdfs;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.io.IOException;
-import java.util.List;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -31,11 +29,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.SocketCache;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 
 import org.apache.hadoop.security.token.Token;
 import org.junit.Test;
@@ -212,6 +210,7 @@ public class TestConnCache {
     MockGetBlockReader answer = new MockGetBlockReader();
     Mockito.doAnswer(answer).when(in).getBlockReader(
                            (InetSocketAddress) Matchers.anyObject(),
+                           (DatanodeInfo) Matchers.anyObject(),
                            Matchers.anyString(),
                            (ExtendedBlock) Matchers.anyObject(),
                            (Token<BlockTokenIdentifier>) Matchers.anyObject(),

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java

@@ -26,11 +26,11 @@ package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import junit.framework.Assert;
 import junit.framework.TestCase;
-import java.net.InetSocketAddress;
-
 import org.apache.hadoop.conf.Configuration;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
@@ -65,9 +65,9 @@ public class TestDFSAddressConfig extends TestCase {
       assertNotNull("Should have been able to stop simulated datanode", dnp);
     }
 
-    conf.unset("dfs.datanode.address");
-    conf.unset("dfs.datanode.http.address");
-    conf.unset("dfs.datanode.ipc.address");
+    conf.unset(DFS_DATANODE_ADDRESS_KEY);
+    conf.unset(DFS_DATANODE_HTTP_ADDRESS_KEY);
+    conf.unset(DFS_DATANODE_IPC_ADDRESS_KEY);
 
     cluster.startDataNodes(conf, 1, true, StartupOption.REGULAR,
                            null, null, null, false, true);
@@ -90,9 +90,9 @@ public class TestDFSAddressConfig extends TestCase {
       assertNotNull("Should have been able to stop simulated datanode", dnp);
     }
 
-    conf.set("dfs.datanode.address","0.0.0.0:0");
-    conf.set("dfs.datanode.http.address","0.0.0.0:0");
-    conf.set("dfs.datanode.ipc.address","0.0.0.0:0");
+    conf.set(DFS_DATANODE_ADDRESS_KEY, "0.0.0.0:0");
+    conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
 
     cluster.startDataNodes(conf, 1, true, StartupOption.REGULAR,
                            null, null, null, false, true);

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

@@ -43,6 +43,7 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileSystem;
@@ -58,10 +59,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
@@ -144,7 +142,7 @@ public class TestDFSClientRetries extends TestCase {
     conf.setInt(DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, 1);
     // set a small buffer size
     final int bufferSize = 4096;
-    conf.setInt("io.file.buffer.size", bufferSize);
+    conf.setInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, bufferSize);
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     

+ 4 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java

@@ -22,14 +22,13 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
 
-import javax.security.auth.login.LoginException;
-
 import junit.framework.AssertionFailedError;
 import junit.framework.TestCase;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -202,7 +201,7 @@ public class TestDFSPermission extends TestCase {
     switch (op) {
     case CREATE:
       FSDataOutputStream out = fs.create(name, permission, true, 
-          conf.getInt("io.file.buffer.size", 4096),
+          conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
           fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
       out.close();
       break;
@@ -520,8 +519,7 @@ public class TestDFSPermission extends TestCase {
     }
 
     /* Perform an operation and verify if the permission checking is correct */
-    void verifyPermission(UserGroupInformation ugi) throws LoginException,
-        IOException {
+    void verifyPermission(UserGroupInformation ugi) throws IOException {
       if (this.ugi != ugi) {
         setRequiredPermissions(ugi);
         this.ugi = ugi;
@@ -564,8 +562,7 @@ public class TestDFSPermission extends TestCase {
     }
 
     /* Set the permissions required to pass the permission checking */
-    protected void setRequiredPermissions(UserGroupInformation ugi)
-        throws IOException {
+    protected void setRequiredPermissions(UserGroupInformation ugi) {
       if (SUPERUSER.equals(ugi)) {
         requiredAncestorPermission = SUPER_MASK;
         requiredParentPermission = SUPER_MASK;

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java

@@ -30,7 +30,6 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;

部分文件因为文件数量过多而无法显示