소스 검색

Merge trunk into HA branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1211749 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 13 년 전
부모
커밋
d9ea5bb489
52개의 변경된 파일8036개의 추가작업 그리고 559개의 파일을 삭제
  1. 7 0
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
  2. 9 0
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
  3. 30 0
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestKerberosAuthenticationHandler.java
  4. 8 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  5. 19 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  6. 1 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
  7. 9 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/HadoopKerberosName.java
  8. 31 28
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
  9. 9 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  10. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  11. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java
  12. 4162 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/ClientDatanodeProtocolProtos.java
  13. 102 277
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/HdfsProtos.java
  14. 30 29
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/InterDatanodeProtocolProtos.java
  15. 813 40
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/NamenodeProtocolProtos.java
  16. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java
  17. 159 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
  18. 136 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  19. 47 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java
  20. 130 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
  21. 114 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
  22. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java
  23. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
  24. 53 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java
  25. 253 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
  26. 270 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
  27. 344 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  28. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java
  29. 19 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Util.java
  30. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckableNameNodeResource.java
  31. 38 34
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  32. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  33. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  34. 34 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
  35. 106 46
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
  36. 81 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java
  37. 122 0
      hadoop-hdfs-project/hadoop-hdfs/src/proto/ClientDatanodeProtocol.proto
  38. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/proto/InterDatanodeProtocol.proto
  39. 21 3
      hadoop-hdfs-project/hadoop-hdfs/src/proto/NamenodeProtocol.proto
  40. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/proto/hdfs.proto
  41. 253 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
  42. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
  43. 105 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java
  44. 177 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
  45. 51 20
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java
  46. 107 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java
  47. 12 3
      hadoop-mapreduce-project/CHANGES.txt
  48. 14 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
  49. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java
  50. 8 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  51. 2 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
  52. 17 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

+ 7 - 0
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java

@@ -55,6 +55,8 @@ import java.util.Set;
  * It does not have a default value.</li>
  * <li>kerberos.keytab: the keytab file containing the credentials for the Kerberos principal.
  * It does not have a default value.</li>
+ * <li>kerberos.name.rules: kerberos names rules to resolve principal names, see 
+ * {@link KerberosName#setRules(String)}</li>
  * </ul>
  */
 public class KerberosAuthenticationHandler implements AuthenticationHandler {
@@ -151,6 +153,11 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
         throw new ServletException("Keytab does not exist: " + keytab);
       }
 
+      String nameRules = config.getProperty(NAME_RULES, null);
+      if (nameRules != null) {
+        KerberosName.setRules(nameRules);
+      }
+      
       Set<Principal> principals = new HashSet<Principal>();
       principals.add(new KerberosPrincipal(principal));
       Subject subject = new Subject(false, principals, new HashSet<Object>(), new HashSet<Object>());

+ 9 - 0
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java

@@ -385,6 +385,15 @@ public class KerberosName {
     rules = parseRules(ruleString);
   }
 
+  /**
+   * Indicates if the name rules have been set.
+   * 
+   * @return if the name rules have been set.
+   */
+  public static boolean hasRulesBeenSet() {
+    return rules != null;
+  }
+  
   static void printRules() throws IOException {
     int i = 0;
     for(Rule r: rules) {

+ 30 - 0
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestKerberosAuthenticationHandler.java

@@ -18,6 +18,7 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
 import junit.framework.TestCase;
 import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.ietf.jgss.GSSContext;
 import org.ietf.jgss.GSSManager;
 import org.ietf.jgss.GSSName;
@@ -59,6 +60,35 @@ public class TestKerberosAuthenticationHandler extends TestCase {
     super.tearDown();
   }
 
+  public void testNameRules() throws Exception {
+    KerberosName kn = new KerberosName(KerberosTestUtils.getServerPrincipal());
+    assertEquals(KerberosTestUtils.getRealm(), kn.getRealm());
+
+    //destroy handler created in setUp()
+    handler.destroy();
+
+    KerberosName.setRules("RULE:[1:$1@$0](.*@FOO)s/@.*//\nDEFAULT");
+    
+    handler = new KerberosAuthenticationHandler();
+    Properties props = new Properties();
+    props.setProperty(KerberosAuthenticationHandler.PRINCIPAL, KerberosTestUtils.getServerPrincipal());
+    props.setProperty(KerberosAuthenticationHandler.KEYTAB, KerberosTestUtils.getKeytabFile());
+    props.setProperty(KerberosAuthenticationHandler.NAME_RULES, "RULE:[1:$1@$0](.*@BAR)s/@.*//\nDEFAULT");
+    try {
+      handler.init(props);
+    } catch (Exception ex) {
+    }
+    kn = new KerberosName("bar@BAR");
+    assertEquals("bar", kn.getShortName());
+    kn = new KerberosName("bar@FOO");
+    try {
+      kn.getShortName();
+      fail();
+    }
+    catch (Exception ex) {      
+    }
+  }
+  
   public void testInit() throws Exception {
     assertEquals(KerberosTestUtils.getServerPrincipal(), handler.getPrincipal());
     assertEquals(KerberosTestUtils.getKeytabFile(), handler.getKeytab());

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

@@ -72,6 +72,8 @@ Trunk (unreleased changes)
     HADOOP-7876. Provided access to encoded key in DelegationKey for
     use in protobuf based RPCs. (suresh)
 
+    HADOOP-7886. Add toString to FileStatus. (SreeHari via jghoman)
+
   BUGS
 
     HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
@@ -115,6 +117,12 @@ Trunk (unreleased changes)
 
     HADOOP-7874. native libs should be under lib/native/ dir. (tucu)
 
+    HADOOP-7887. KerberosAuthenticatorHandler is not setting
+    KerberosName name rules from configuration. (tucu)
+
+    HADOOP-7888. TestFailoverProxy fails intermittently on trunk. (Jason Lowe
+                 via atm)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)

+ 19 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -331,4 +331,23 @@ public class FileStatus implements Writable, Comparable {
   public int hashCode() {
     return getPath().hashCode();
   }
+  
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName()); 
+    sb.append("{");
+    sb.append("path=" + path);
+    sb.append("; isDirectory=" + isdir);
+    if(!isDirectory()){
+      sb.append("; length=" + length);
+      sb.append("; replication=" + block_replication);
+      sb.append("; blocksize=" + blocksize);
+    }
+    sb.append("; owner=" + owner);
+    sb.append("; group=" + group);
+    sb.append("; permission=" + permission);
+    sb.append("}");
+    return sb.toString();
+  }
 }

+ 1 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java

@@ -103,15 +103,12 @@ class RetryInvocationHandler implements InvocationHandler, Closeable {
             if (invocationAttemptFailoverCount == proxyProviderFailoverCount) {
               proxyProvider.performFailover(currentProxy);
               proxyProviderFailoverCount++;
+              currentProxy = proxyProvider.getProxy();
             } else {
               LOG.warn("A failover has occurred since the start of this method"
                   + " invocation attempt.");
             }
           }
-          // The call to getProxy() could technically only be made in the event
-          // performFailover() is called, but it needs to be out here for the
-          // purpose of testing.
-          currentProxy = proxyProvider.getProxy();
           invocationFailoverCount++;
         }
         if(LOG.isDebugEnabled()) {

+ 9 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/HadoopKerberosName.java

@@ -56,12 +56,19 @@ public class HadoopKerberosName extends KerberosName {
   }
   /**
    * Set the static configuration to get the rules.
+   * <p/>
+   * IMPORTANT: This method does a NOP if the rules have been set already.
+   * If there is a need to reset the rules, the {@link KerberosName#setRules(String)}
+   * method should be invoked directly.
+   * 
    * @param conf the new configuration
    * @throws IOException
    */
   public static void setConfiguration(Configuration conf) throws IOException {
-    String ruleString = conf.get("hadoop.security.auth_to_local", "DEFAULT");
-    setRules(ruleString);
+    if (!hasRulesBeenSet()) {
+      String ruleString = conf.get("hadoop.security.auth_to_local", "DEFAULT");
+      setRules(ruleString);
+    }
   }
 
   public static void main(String[] args) throws Exception {

+ 31 - 28
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java

@@ -36,34 +36,18 @@ public class TestFailoverProxy {
     private Object impl1;
     private Object impl2;
     
-    private boolean latchEnabled = false;
-    private CountDownLatch getProxyLatch;
     private int failoversOccurred = 0;
     
     public FlipFlopProxyProvider(Class<?> iface, Object activeImpl,
-        Object standbyImpl, int getProxyCountDown) {
+        Object standbyImpl) {
       this.iface = iface;
       this.impl1 = activeImpl;
       this.impl2 = standbyImpl;
       currentlyActive = impl1;
-      getProxyLatch = new CountDownLatch(getProxyCountDown);
-    }
-    
-    public FlipFlopProxyProvider(Class<?> iface, Object activeImpl,
-        Object standbyImpl) {
-      this(iface, activeImpl, standbyImpl, 0);
     }
     
     @Override
     public Object getProxy() {
-      if (latchEnabled) {
-        getProxyLatch.countDown();
-        try {
-          getProxyLatch.await();
-        } catch (InterruptedException e) {
-          throw new RuntimeException(e);
-        }
-      }
       return currentlyActive;
     }
 
@@ -83,10 +67,6 @@ public class TestFailoverProxy {
       // Nothing to do.
     }
     
-    public void setLatchEnabled(boolean latchEnabled) {
-      this.latchEnabled = latchEnabled;
-    }
-    
     public int getFailoversOccurred() {
       return failoversOccurred;
     }
@@ -214,6 +194,32 @@ public class TestFailoverProxy {
     assertEquals("impl2", unreliable.succeedsOnceThenFailsReturningStringIdempotent());
   }
   
+  private static class SynchronizedUnreliableImplementation extends UnreliableImplementation {
+    
+    private CountDownLatch methodLatch;
+    
+    public SynchronizedUnreliableImplementation(String identifier,
+        TypeOfExceptionToFailWith exceptionToFailWith, int threadCount) {
+      super(identifier, exceptionToFailWith);
+      
+      methodLatch = new CountDownLatch(threadCount);
+    }
+
+    @Override
+    public String failsIfIdentifierDoesntMatch(String identifier)
+        throws UnreliableException, StandbyException, IOException {
+      // Wait until all threads are trying to invoke this method
+      methodLatch.countDown();
+      try {
+        methodLatch.await();
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+      return super.failsIfIdentifierDoesntMatch(identifier);
+    }
+    
+  }
+  
   private static class ConcurrentMethodThread extends Thread {
     
     private UnreliableInterface unreliable;
@@ -240,11 +246,11 @@ public class TestFailoverProxy {
   public void testConcurrentMethodFailures() throws InterruptedException {
     FlipFlopProxyProvider proxyProvider = new FlipFlopProxyProvider(
         UnreliableInterface.class,
-        new UnreliableImplementation("impl1",
-            TypeOfExceptionToFailWith.STANDBY_EXCEPTION),
+        new SynchronizedUnreliableImplementation("impl1",
+            TypeOfExceptionToFailWith.STANDBY_EXCEPTION,
+            2),
         new UnreliableImplementation("impl2",
-            TypeOfExceptionToFailWith.STANDBY_EXCEPTION),
-        2);
+            TypeOfExceptionToFailWith.STANDBY_EXCEPTION));
     
     final UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
       .create(UnreliableInterface.class, proxyProvider,
@@ -253,9 +259,6 @@ public class TestFailoverProxy {
     ConcurrentMethodThread t1 = new ConcurrentMethodThread(unreliable);
     ConcurrentMethodThread t2 = new ConcurrentMethodThread(unreliable);
     
-    // Getting a proxy will now wait on a latch.
-    proxyProvider.setLatchEnabled(true);
-    
     t1.start();
     t2.start();
     t1.join();

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

@@ -15,6 +15,15 @@ Trunk (unreleased changes)
 
     HDFS-2581. Implement protobuf service for JournalProtocol. (suresh)
 
+    HDFS-2618. Implement protobuf service for NamenodeProtocol. (suresh)
+
+    HDFS-2629. Implement protobuf service for InterDatanodeProtocol. (suresh)
+
+    HDFS-2636. Implement protobuf service for ClientDatanodeProtocol. (suresh)
+
+    HDFS-2430. The number of failed or low-resource volumes the NN can tolerate
+               should be configurable. (atm)
+
   IMPROVEMENTS
 
     HADOOP-7524 Change RPC to allow multiple protocols including multuple 

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

@@ -130,6 +130,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_DEFAULT = true;
   public static final String  DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY = "dfs.namenode.num.checkpoints.retained";
   public static final int     DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_DEFAULT = 2;
+  public static final String  DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
+  public static final int     DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
   
   public static final String  DFS_LIST_LIMIT = "dfs.ls.limit";
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;
@@ -164,6 +166,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_EDITS_DIR_KEY = "dfs.namenode.edits.dir";
   public static final String  DFS_NAMENODE_SHARED_EDITS_DIR_KEY = "dfs.namenode.shared.edits.dir";
   public static final String  DFS_NAMENODE_EDITS_PLUGIN_PREFIX = "dfs.namenode.edits.journal-plugin";
+  public static final String  DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY = "dfs.namenode.edits.dir.required";
   public static final String  DFS_CLIENT_READ_PREFETCH_SIZE_KEY = "dfs.client.read.prefetch.size"; 
   public static final String  DFS_CLIENT_RETRY_WINDOW_BASE= "dfs.client.retry.window.base";
   public static final String  DFS_METRICS_SESSION_ID_KEY = "dfs.metrics.session-id";
@@ -306,6 +309,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_DU_RESERVED_KEY = "dfs.namenode.resource.du.reserved";
   public static final long    DFS_NAMENODE_DU_RESERVED_DEFAULT = 1024 * 1024 * 100; // 100 MB
   public static final String  DFS_NAMENODE_CHECKED_VOLUMES_KEY = "dfs.namenode.resource.checked.volumes";
+  public static final String  DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_KEY = "dfs.namenode.resource.checked.volumes.minimum";
+  public static final int     DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_DEFAULT = 1;
   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";
   

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

@@ -66,6 +66,11 @@ public class BlockLocalPathInfo implements Writable {
    */
   public String getBlockPath() {return localBlockPath;}
   
+  /**
+   * @return the Block
+   */
+  public ExtendedBlock getBlock() { return block;}
+  
   /**
    * Get the Block metadata file.
    * @return Block metadata file.

+ 4162 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/ClientDatanodeProtocolProtos.java

@@ -0,0 +1,4162 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: ClientDatanodeProtocol.proto
+
+package org.apache.hadoop.hdfs.protocol.proto;
+
+public final class ClientDatanodeProtocolProtos {
+  private ClientDatanodeProtocolProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface GetReplicaVisibleLengthRequestProtoOrBuilder
+      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 GetReplicaVisibleLengthRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetReplicaVisibleLengthRequestProtoOrBuilder {
+    // Use GetReplicaVisibleLengthRequestProto.newBuilder() to construct.
+    private GetReplicaVisibleLengthRequestProto(Builder builder) {
+      super(builder);
+    }
+    private GetReplicaVisibleLengthRequestProto(boolean noInit) {}
+    
+    private static final GetReplicaVisibleLengthRequestProto defaultInstance;
+    public static GetReplicaVisibleLengthRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetReplicaVisibleLengthRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetReplicaVisibleLengthRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetReplicaVisibleLengthRequestProto_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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto) 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetReplicaVisibleLengthRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetReplicaVisibleLengthRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto.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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto(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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto.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:GetReplicaVisibleLengthRequestProto)
+    }
+    
+    static {
+      defaultInstance = new GetReplicaVisibleLengthRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetReplicaVisibleLengthRequestProto)
+  }
+  
+  public interface GetReplicaVisibleLengthResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required uint64 length = 1;
+    boolean hasLength();
+    long getLength();
+  }
+  public static final class GetReplicaVisibleLengthResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetReplicaVisibleLengthResponseProtoOrBuilder {
+    // Use GetReplicaVisibleLengthResponseProto.newBuilder() to construct.
+    private GetReplicaVisibleLengthResponseProto(Builder builder) {
+      super(builder);
+    }
+    private GetReplicaVisibleLengthResponseProto(boolean noInit) {}
+    
+    private static final GetReplicaVisibleLengthResponseProto defaultInstance;
+    public static GetReplicaVisibleLengthResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetReplicaVisibleLengthResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetReplicaVisibleLengthResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetReplicaVisibleLengthResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required uint64 length = 1;
+    public static final int LENGTH_FIELD_NUMBER = 1;
+    private long length_;
+    public boolean hasLength() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public long getLength() {
+      return length_;
+    }
+    
+    private void initFields() {
+      length_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasLength()) {
+        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, length_);
+      }
+      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, length_);
+      }
+      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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasLength() == other.hasLength());
+      if (hasLength()) {
+        result = result && (getLength()
+            == other.getLength());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasLength()) {
+        hash = (37 * hash) + LENGTH_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getLength());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetReplicaVisibleLengthResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetReplicaVisibleLengthResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.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();
+        length_ = 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.length_ = length_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.getDefaultInstance()) return this;
+        if (other.hasLength()) {
+          setLength(other.getLength());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasLength()) {
+          
+          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;
+              length_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required uint64 length = 1;
+      private long length_ ;
+      public boolean hasLength() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public long getLength() {
+        return length_;
+      }
+      public Builder setLength(long value) {
+        bitField0_ |= 0x00000001;
+        length_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearLength() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        length_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:GetReplicaVisibleLengthResponseProto)
+    }
+    
+    static {
+      defaultInstance = new GetReplicaVisibleLengthResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetReplicaVisibleLengthResponseProto)
+  }
+  
+  public interface RefreshNamenodesRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class RefreshNamenodesRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements RefreshNamenodesRequestProtoOrBuilder {
+    // Use RefreshNamenodesRequestProto.newBuilder() to construct.
+    private RefreshNamenodesRequestProto(Builder builder) {
+      super(builder);
+    }
+    private RefreshNamenodesRequestProto(boolean noInit) {}
+    
+    private static final RefreshNamenodesRequestProto defaultInstance;
+    public static RefreshNamenodesRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public RefreshNamenodesRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_RefreshNamenodesRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_RefreshNamenodesRequestProto_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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto) 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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_RefreshNamenodesRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_RefreshNamenodesRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto.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.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto.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:RefreshNamenodesRequestProto)
+    }
+    
+    static {
+      defaultInstance = new RefreshNamenodesRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:RefreshNamenodesRequestProto)
+  }
+  
+  public interface RefreshNamenodesResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class RefreshNamenodesResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements RefreshNamenodesResponseProtoOrBuilder {
+    // Use RefreshNamenodesResponseProto.newBuilder() to construct.
+    private RefreshNamenodesResponseProto(Builder builder) {
+      super(builder);
+    }
+    private RefreshNamenodesResponseProto(boolean noInit) {}
+    
+    private static final RefreshNamenodesResponseProto defaultInstance;
+    public static RefreshNamenodesResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public RefreshNamenodesResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_RefreshNamenodesResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_RefreshNamenodesResponseProto_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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto) 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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto 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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_RefreshNamenodesResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_RefreshNamenodesResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.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.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.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:RefreshNamenodesResponseProto)
+    }
+    
+    static {
+      defaultInstance = new RefreshNamenodesResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:RefreshNamenodesResponseProto)
+  }
+  
+  public interface DeleteBlockPoolRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required string blockPool = 1;
+    boolean hasBlockPool();
+    String getBlockPool();
+    
+    // required bool force = 2;
+    boolean hasForce();
+    boolean getForce();
+  }
+  public static final class DeleteBlockPoolRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements DeleteBlockPoolRequestProtoOrBuilder {
+    // Use DeleteBlockPoolRequestProto.newBuilder() to construct.
+    private DeleteBlockPoolRequestProto(Builder builder) {
+      super(builder);
+    }
+    private DeleteBlockPoolRequestProto(boolean noInit) {}
+    
+    private static final DeleteBlockPoolRequestProto defaultInstance;
+    public static DeleteBlockPoolRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public DeleteBlockPoolRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_DeleteBlockPoolRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_DeleteBlockPoolRequestProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required string blockPool = 1;
+    public static final int BLOCKPOOL_FIELD_NUMBER = 1;
+    private java.lang.Object blockPool_;
+    public boolean hasBlockPool() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public String getBlockPool() {
+      java.lang.Object ref = blockPool_;
+      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)) {
+          blockPool_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getBlockPoolBytes() {
+      java.lang.Object ref = blockPool_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        blockPool_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // required bool force = 2;
+    public static final int FORCE_FIELD_NUMBER = 2;
+    private boolean force_;
+    public boolean hasForce() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public boolean getForce() {
+      return force_;
+    }
+    
+    private void initFields() {
+      blockPool_ = "";
+      force_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlockPool()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasForce()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getBlockPoolBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, force_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getBlockPoolBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, force_);
+      }
+      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.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlockPool() == other.hasBlockPool());
+      if (hasBlockPool()) {
+        result = result && getBlockPool()
+            .equals(other.getBlockPool());
+      }
+      result = result && (hasForce() == other.hasForce());
+      if (hasForce()) {
+        result = result && (getForce()
+            == other.getForce());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasBlockPool()) {
+        hash = (37 * hash) + BLOCKPOOL_FIELD_NUMBER;
+        hash = (53 * hash) + getBlockPool().hashCode();
+      }
+      if (hasForce()) {
+        hash = (37 * hash) + FORCE_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getForce());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_DeleteBlockPoolRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_DeleteBlockPoolRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto.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();
+        blockPool_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        force_ = false;
+        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.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.blockPool_ = blockPool_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.force_ = force_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto.getDefaultInstance()) return this;
+        if (other.hasBlockPool()) {
+          setBlockPool(other.getBlockPool());
+        }
+        if (other.hasForce()) {
+          setForce(other.getForce());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBlockPool()) {
+          
+          return false;
+        }
+        if (!hasForce()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              blockPool_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              force_ = input.readBool();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required string blockPool = 1;
+      private java.lang.Object blockPool_ = "";
+      public boolean hasBlockPool() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public String getBlockPool() {
+        java.lang.Object ref = blockPool_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          blockPool_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setBlockPool(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        blockPool_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearBlockPool() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        blockPool_ = getDefaultInstance().getBlockPool();
+        onChanged();
+        return this;
+      }
+      void setBlockPool(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000001;
+        blockPool_ = value;
+        onChanged();
+      }
+      
+      // required bool force = 2;
+      private boolean force_ ;
+      public boolean hasForce() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public boolean getForce() {
+        return force_;
+      }
+      public Builder setForce(boolean value) {
+        bitField0_ |= 0x00000002;
+        force_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearForce() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        force_ = false;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:DeleteBlockPoolRequestProto)
+    }
+    
+    static {
+      defaultInstance = new DeleteBlockPoolRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:DeleteBlockPoolRequestProto)
+  }
+  
+  public interface DeleteBlockPoolResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  public static final class DeleteBlockPoolResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements DeleteBlockPoolResponseProtoOrBuilder {
+    // Use DeleteBlockPoolResponseProto.newBuilder() to construct.
+    private DeleteBlockPoolResponseProto(Builder builder) {
+      super(builder);
+    }
+    private DeleteBlockPoolResponseProto(boolean noInit) {}
+    
+    private static final DeleteBlockPoolResponseProto defaultInstance;
+    public static DeleteBlockPoolResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public DeleteBlockPoolResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_DeleteBlockPoolResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_DeleteBlockPoolResponseProto_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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto) 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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto 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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_DeleteBlockPoolResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_DeleteBlockPoolResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.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.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto(this);
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.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:DeleteBlockPoolResponseProto)
+    }
+    
+    static {
+      defaultInstance = new DeleteBlockPoolResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:DeleteBlockPoolResponseProto)
+  }
+  
+  public interface GetBlockLocalPathInfoRequestProtoOrBuilder
+      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 .BlockTokenIdentifierProto token = 2;
+    boolean hasToken();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto getToken();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder getTokenOrBuilder();
+  }
+  public static final class GetBlockLocalPathInfoRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetBlockLocalPathInfoRequestProtoOrBuilder {
+    // Use GetBlockLocalPathInfoRequestProto.newBuilder() to construct.
+    private GetBlockLocalPathInfoRequestProto(Builder builder) {
+      super(builder);
+    }
+    private GetBlockLocalPathInfoRequestProto(boolean noInit) {}
+    
+    private static final GetBlockLocalPathInfoRequestProto defaultInstance;
+    public static GetBlockLocalPathInfoRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetBlockLocalPathInfoRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetBlockLocalPathInfoRequestProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetBlockLocalPathInfoRequestProto_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 .BlockTokenIdentifierProto token = 2;
+    public static final int TOKEN_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto token_;
+    public boolean hasToken() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto getToken() {
+      return token_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder getTokenOrBuilder() {
+      return token_;
+    }
+    
+    private void initFields() {
+      block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+      token_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasToken()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getBlock().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getToken().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, block_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, token_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, block_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, token_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto other = (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlock() == other.hasBlock());
+      if (hasBlock()) {
+        result = result && getBlock()
+            .equals(other.getBlock());
+      }
+      result = result && (hasToken() == other.hasToken());
+      if (hasToken()) {
+        result = result && getToken()
+            .equals(other.getToken());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasBlock()) {
+        hash = (37 * hash) + BLOCK_FIELD_NUMBER;
+        hash = (53 * hash) + getBlock().hashCode();
+      }
+      if (hasToken()) {
+        hash = (37 * hash) + TOKEN_FIELD_NUMBER;
+        hash = (53 * hash) + getToken().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetBlockLocalPathInfoRequestProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetBlockLocalPathInfoRequestProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBlockFieldBuilder();
+          getTokenFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (blockBuilder_ == null) {
+          block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (tokenBuilder_ == null) {
+          token_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance();
+        } else {
+          tokenBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto result = new org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (blockBuilder_ == null) {
+          result.block_ = block_;
+        } else {
+          result.block_ = blockBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tokenBuilder_ == null) {
+          result.token_ = token_;
+        } else {
+          result.token_ = tokenBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto.getDefaultInstance()) return this;
+        if (other.hasBlock()) {
+          mergeBlock(other.getBlock());
+        }
+        if (other.hasToken()) {
+          mergeToken(other.getToken());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBlock()) {
+          
+          return false;
+        }
+        if (!hasToken()) {
+          
+          return false;
+        }
+        if (!getBlock().isInitialized()) {
+          
+          return false;
+        }
+        if (!getToken().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.newBuilder();
+              if (hasBlock()) {
+                subBuilder.mergeFrom(getBlock());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setBlock(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.newBuilder();
+              if (hasToken()) {
+                subBuilder.mergeFrom(getToken());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setToken(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .ExtendedBlockProto block = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder> blockBuilder_;
+      public boolean hasBlock() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto getBlock() {
+        if (blockBuilder_ == null) {
+          return block_;
+        } else {
+          return blockBuilder_.getMessage();
+        }
+      }
+      public Builder setBlock(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto value) {
+        if (blockBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          block_ = value;
+          onChanged();
+        } else {
+          blockBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setBlock(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder builderForValue) {
+        if (blockBuilder_ == null) {
+          block_ = builderForValue.build();
+          onChanged();
+        } else {
+          blockBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeBlock(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto value) {
+        if (blockBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              block_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance()) {
+            block_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.newBuilder(block_).mergeFrom(value).buildPartial();
+          } else {
+            block_ = value;
+          }
+          onChanged();
+        } else {
+          blockBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearBlock() {
+        if (blockBuilder_ == null) {
+          block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+          onChanged();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder getBlockBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getBlockFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder getBlockOrBuilder() {
+        if (blockBuilder_ != null) {
+          return blockBuilder_.getMessageOrBuilder();
+        } else {
+          return block_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder> 
+          getBlockFieldBuilder() {
+        if (blockBuilder_ == null) {
+          blockBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder>(
+                  block_,
+                  getParentForChildren(),
+                  isClean());
+          block_ = null;
+        }
+        return blockBuilder_;
+      }
+      
+      // required .BlockTokenIdentifierProto token = 2;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto token_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder> tokenBuilder_;
+      public boolean hasToken() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto getToken() {
+        if (tokenBuilder_ == null) {
+          return token_;
+        } else {
+          return tokenBuilder_.getMessage();
+        }
+      }
+      public Builder setToken(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto value) {
+        if (tokenBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          token_ = value;
+          onChanged();
+        } else {
+          tokenBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder setToken(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder builderForValue) {
+        if (tokenBuilder_ == null) {
+          token_ = builderForValue.build();
+          onChanged();
+        } else {
+          tokenBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder mergeToken(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto value) {
+        if (tokenBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              token_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance()) {
+            token_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.newBuilder(token_).mergeFrom(value).buildPartial();
+          } else {
+            token_ = value;
+          }
+          onChanged();
+        } else {
+          tokenBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder clearToken() {
+        if (tokenBuilder_ == null) {
+          token_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance();
+          onChanged();
+        } else {
+          tokenBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder getTokenBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTokenFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder getTokenOrBuilder() {
+        if (tokenBuilder_ != null) {
+          return tokenBuilder_.getMessageOrBuilder();
+        } else {
+          return token_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder> 
+          getTokenFieldBuilder() {
+        if (tokenBuilder_ == null) {
+          tokenBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder>(
+                  token_,
+                  getParentForChildren(),
+                  isClean());
+          token_ = null;
+        }
+        return tokenBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:GetBlockLocalPathInfoRequestProto)
+    }
+    
+    static {
+      defaultInstance = new GetBlockLocalPathInfoRequestProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetBlockLocalPathInfoRequestProto)
+  }
+  
+  public interface GetBlockLocalPathInfoResponseProtoOrBuilder
+      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 string localPath = 2;
+    boolean hasLocalPath();
+    String getLocalPath();
+    
+    // required string localMetaPath = 3;
+    boolean hasLocalMetaPath();
+    String getLocalMetaPath();
+  }
+  public static final class GetBlockLocalPathInfoResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements GetBlockLocalPathInfoResponseProtoOrBuilder {
+    // Use GetBlockLocalPathInfoResponseProto.newBuilder() to construct.
+    private GetBlockLocalPathInfoResponseProto(Builder builder) {
+      super(builder);
+    }
+    private GetBlockLocalPathInfoResponseProto(boolean noInit) {}
+    
+    private static final GetBlockLocalPathInfoResponseProto defaultInstance;
+    public static GetBlockLocalPathInfoResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public GetBlockLocalPathInfoResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetBlockLocalPathInfoResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetBlockLocalPathInfoResponseProto_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 string localPath = 2;
+    public static final int LOCALPATH_FIELD_NUMBER = 2;
+    private java.lang.Object localPath_;
+    public boolean hasLocalPath() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public String getLocalPath() {
+      java.lang.Object ref = localPath_;
+      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)) {
+          localPath_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getLocalPathBytes() {
+      java.lang.Object ref = localPath_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        localPath_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // required string localMetaPath = 3;
+    public static final int LOCALMETAPATH_FIELD_NUMBER = 3;
+    private java.lang.Object localMetaPath_;
+    public boolean hasLocalMetaPath() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public String getLocalMetaPath() {
+      java.lang.Object ref = localMetaPath_;
+      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)) {
+          localMetaPath_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getLocalMetaPathBytes() {
+      java.lang.Object ref = localMetaPath_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        localMetaPath_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    private void initFields() {
+      block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+      localPath_ = "";
+      localMetaPath_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasLocalPath()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasLocalMetaPath()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getBlock().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, block_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getLocalPathBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, getLocalMetaPathBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, block_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getLocalPathBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getLocalMetaPathBytes());
+      }
+      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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlock() == other.hasBlock());
+      if (hasBlock()) {
+        result = result && getBlock()
+            .equals(other.getBlock());
+      }
+      result = result && (hasLocalPath() == other.hasLocalPath());
+      if (hasLocalPath()) {
+        result = result && getLocalPath()
+            .equals(other.getLocalPath());
+      }
+      result = result && (hasLocalMetaPath() == other.hasLocalMetaPath());
+      if (hasLocalMetaPath()) {
+        result = result && getLocalMetaPath()
+            .equals(other.getLocalMetaPath());
+      }
+      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 (hasLocalPath()) {
+        hash = (37 * hash) + LOCALPATH_FIELD_NUMBER;
+        hash = (53 * hash) + getLocalPath().hashCode();
+      }
+      if (hasLocalMetaPath()) {
+        hash = (37 * hash) + LOCALMETAPATH_FIELD_NUMBER;
+        hash = (53 * hash) + getLocalMetaPath().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto 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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetBlockLocalPathInfoResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.internal_static_GetBlockLocalPathInfoResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.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);
+        localPath_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        localMetaPath_ = "";
+        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.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto(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.localPath_ = localPath_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.localMetaPath_ = localMetaPath_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.getDefaultInstance()) return this;
+        if (other.hasBlock()) {
+          mergeBlock(other.getBlock());
+        }
+        if (other.hasLocalPath()) {
+          setLocalPath(other.getLocalPath());
+        }
+        if (other.hasLocalMetaPath()) {
+          setLocalMetaPath(other.getLocalMetaPath());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBlock()) {
+          
+          return false;
+        }
+        if (!hasLocalPath()) {
+          
+          return false;
+        }
+        if (!hasLocalMetaPath()) {
+          
+          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 18: {
+              bitField0_ |= 0x00000002;
+              localPath_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              localMetaPath_ = input.readBytes();
+              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 string localPath = 2;
+      private java.lang.Object localPath_ = "";
+      public boolean hasLocalPath() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public String getLocalPath() {
+        java.lang.Object ref = localPath_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          localPath_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setLocalPath(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        localPath_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearLocalPath() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        localPath_ = getDefaultInstance().getLocalPath();
+        onChanged();
+        return this;
+      }
+      void setLocalPath(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000002;
+        localPath_ = value;
+        onChanged();
+      }
+      
+      // required string localMetaPath = 3;
+      private java.lang.Object localMetaPath_ = "";
+      public boolean hasLocalMetaPath() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public String getLocalMetaPath() {
+        java.lang.Object ref = localMetaPath_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          localMetaPath_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setLocalMetaPath(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        localMetaPath_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearLocalMetaPath() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        localMetaPath_ = getDefaultInstance().getLocalMetaPath();
+        onChanged();
+        return this;
+      }
+      void setLocalMetaPath(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000004;
+        localMetaPath_ = value;
+        onChanged();
+      }
+      
+      // @@protoc_insertion_point(builder_scope:GetBlockLocalPathInfoResponseProto)
+    }
+    
+    static {
+      defaultInstance = new GetBlockLocalPathInfoResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:GetBlockLocalPathInfoResponseProto)
+  }
+  
+  public static abstract class ClientDatanodeProtocolService
+      implements com.google.protobuf.Service {
+    protected ClientDatanodeProtocolService() {}
+    
+    public interface Interface {
+      public abstract void getReplicaVisibleLength(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto> done);
+      
+      public abstract void refreshNamenode(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto> done);
+      
+      public abstract void deleteBlockPool(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto> done);
+      
+      public abstract void getBlockLocalPathInfo(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto> done);
+      
+    }
+    
+    public static com.google.protobuf.Service newReflectiveService(
+        final Interface impl) {
+      return new ClientDatanodeProtocolService() {
+        @java.lang.Override
+        public  void getReplicaVisibleLength(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto> done) {
+          impl.getReplicaVisibleLength(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void refreshNamenode(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto> done) {
+          impl.refreshNamenode(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void deleteBlockPool(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto> done) {
+          impl.deleteBlockPool(controller, request, done);
+        }
+        
+        @java.lang.Override
+        public  void getBlockLocalPathInfo(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto> done) {
+          impl.getBlockLocalPathInfo(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.getReplicaVisibleLength(controller, (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto)request);
+            case 1:
+              return impl.refreshNamenode(controller, (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto)request);
+            case 2:
+              return impl.deleteBlockPool(controller, (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto)request);
+            case 3:
+              return impl.getBlockLocalPathInfo(controller, (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto)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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto.getDefaultInstance();
+            case 2:
+              return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto.getDefaultInstance();
+            case 3:
+              return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto.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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.getDefaultInstance();
+            case 1:
+              return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.getDefaultInstance();
+            case 2:
+              return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.getDefaultInstance();
+            case 3:
+              return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+      };
+    }
+    
+    public abstract void getReplicaVisibleLength(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto> done);
+    
+    public abstract void refreshNamenode(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto> done);
+    
+    public abstract void deleteBlockPool(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto> done);
+    
+    public abstract void getBlockLocalPathInfo(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto> done);
+    
+    public static final
+        com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.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.getReplicaVisibleLength(controller, (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto>specializeCallback(
+              done));
+          return;
+        case 1:
+          this.refreshNamenode(controller, (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto>specializeCallback(
+              done));
+          return;
+        case 2:
+          this.deleteBlockPool(controller, (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto>specializeCallback(
+              done));
+          return;
+        case 3:
+          this.getBlockLocalPathInfo(controller, (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto>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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto.getDefaultInstance();
+        case 2:
+          return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto.getDefaultInstance();
+        case 3:
+          return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto.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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.getDefaultInstance();
+        case 1:
+          return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.getDefaultInstance();
+        case 2:
+          return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.getDefaultInstance();
+        case 3:
+          return org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.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.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService 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 getReplicaVisibleLength(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.getDefaultInstance()));
+      }
+      
+      public  void refreshNamenode(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.getDefaultInstance()));
+      }
+      
+      public  void deleteBlockPool(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(2),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.getDefaultInstance()));
+      }
+      
+      public  void getBlockLocalPathInfo(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(3),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.class,
+            org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.getDefaultInstance()));
+      }
+    }
+    
+    public static BlockingInterface newBlockingStub(
+        com.google.protobuf.BlockingRpcChannel channel) {
+      return new BlockingStub(channel);
+    }
+    
+    public interface BlockingInterface {
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto getReplicaVisibleLength(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto refreshNamenode(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto deleteBlockPool(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto request)
+          throws com.google.protobuf.ServiceException;
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto getBlockLocalPathInfo(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto 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.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto getReplicaVisibleLength(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto refreshNamenode(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto deleteBlockPool(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(2),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.getDefaultInstance());
+      }
+      
+      
+      public org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto getBlockLocalPathInfo(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(3),
+          controller,
+          request,
+          org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.getDefaultInstance());
+      }
+      
+    }
+  }
+  
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetReplicaVisibleLengthRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetReplicaVisibleLengthRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetReplicaVisibleLengthResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetReplicaVisibleLengthResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_RefreshNamenodesRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_RefreshNamenodesRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_RefreshNamenodesResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_RefreshNamenodesResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DeleteBlockPoolRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DeleteBlockPoolRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DeleteBlockPoolResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DeleteBlockPoolResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetBlockLocalPathInfoRequestProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetBlockLocalPathInfoRequestProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_GetBlockLocalPathInfoResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_GetBlockLocalPathInfoResponseProto_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\034ClientDatanodeProtocol.proto\032\nhdfs.pro" +
+      "to\"I\n#GetReplicaVisibleLengthRequestProt" +
+      "o\022\"\n\005block\030\001 \002(\0132\023.ExtendedBlockProto\"6\n" +
+      "$GetReplicaVisibleLengthResponseProto\022\016\n" +
+      "\006length\030\001 \002(\004\"\036\n\034RefreshNamenodesRequest" +
+      "Proto\"\037\n\035RefreshNamenodesResponseProto\"?" +
+      "\n\033DeleteBlockPoolRequestProto\022\021\n\tblockPo" +
+      "ol\030\001 \002(\t\022\r\n\005force\030\002 \002(\010\"\036\n\034DeleteBlockPo" +
+      "olResponseProto\"r\n!GetBlockLocalPathInfo" +
+      "RequestProto\022\"\n\005block\030\001 \002(\0132\023.ExtendedBl",
+      "ockProto\022)\n\005token\030\002 \002(\0132\032.BlockTokenIden" +
+      "tifierProto\"r\n\"GetBlockLocalPathInfoResp" +
+      "onseProto\022\"\n\005block\030\001 \002(\0132\023.ExtendedBlock" +
+      "Proto\022\021\n\tlocalPath\030\002 \002(\t\022\025\n\rlocalMetaPat" +
+      "h\030\003 \002(\t2\213\003\n\035ClientDatanodeProtocolServic" +
+      "e\022f\n\027getReplicaVisibleLength\022$.GetReplic" +
+      "aVisibleLengthRequestProto\032%.GetReplicaV" +
+      "isibleLengthResponseProto\022P\n\017refreshName" +
+      "node\022\035.RefreshNamenodesRequestProto\032\036.Re" +
+      "freshNamenodesResponseProto\022N\n\017deleteBlo",
+      "ckPool\022\034.DeleteBlockPoolRequestProto\032\035.D" +
+      "eleteBlockPoolResponseProto\022`\n\025getBlockL" +
+      "ocalPathInfo\022\".GetBlockLocalPathInfoRequ" +
+      "estProto\032#.GetBlockLocalPathInfoResponse" +
+      "ProtoBK\n%org.apache.hadoop.hdfs.protocol" +
+      ".protoB\034ClientDatanodeProtocolProtos\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_GetReplicaVisibleLengthRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_GetReplicaVisibleLengthRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetReplicaVisibleLengthRequestProto_descriptor,
+              new java.lang.String[] { "Block", },
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto.Builder.class);
+          internal_static_GetReplicaVisibleLengthResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_GetReplicaVisibleLengthResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetReplicaVisibleLengthResponseProto_descriptor,
+              new java.lang.String[] { "Length", },
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto.Builder.class);
+          internal_static_RefreshNamenodesRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(2);
+          internal_static_RefreshNamenodesRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_RefreshNamenodesRequestProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto.Builder.class);
+          internal_static_RefreshNamenodesResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(3);
+          internal_static_RefreshNamenodesResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_RefreshNamenodesResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto.Builder.class);
+          internal_static_DeleteBlockPoolRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(4);
+          internal_static_DeleteBlockPoolRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_DeleteBlockPoolRequestProto_descriptor,
+              new java.lang.String[] { "BlockPool", "Force", },
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto.Builder.class);
+          internal_static_DeleteBlockPoolResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(5);
+          internal_static_DeleteBlockPoolResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_DeleteBlockPoolResponseProto_descriptor,
+              new java.lang.String[] { },
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto.Builder.class);
+          internal_static_GetBlockLocalPathInfoRequestProto_descriptor =
+            getDescriptor().getMessageTypes().get(6);
+          internal_static_GetBlockLocalPathInfoRequestProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetBlockLocalPathInfoRequestProto_descriptor,
+              new java.lang.String[] { "Block", "Token", },
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto.Builder.class);
+          internal_static_GetBlockLocalPathInfoResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(7);
+          internal_static_GetBlockLocalPathInfoResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_GetBlockLocalPathInfoResponseProto_descriptor,
+              new java.lang.String[] { "Block", "LocalPath", "LocalMetaPath", },
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto.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)
+}

+ 102 - 277
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/HdfsProtos.java

@@ -8,7 +8,7 @@ public final class HdfsProtos {
   public static void registerAllExtensions(
       com.google.protobuf.ExtensionRegistry registry) {
   }
-  public enum ReplicaState
+  public enum ReplicaStateProto
       implements com.google.protobuf.ProtocolMessageEnum {
     FINALIZED(0, 0),
     RBW(1, 1),
@@ -26,7 +26,7 @@ public final class HdfsProtos {
     
     public final int getNumber() { return value; }
     
-    public static ReplicaState valueOf(int value) {
+    public static ReplicaStateProto valueOf(int value) {
       switch (value) {
         case 0: return FINALIZED;
         case 1: return RBW;
@@ -37,15 +37,15 @@ public final class HdfsProtos {
       }
     }
     
-    public static com.google.protobuf.Internal.EnumLiteMap<ReplicaState>
+    public static com.google.protobuf.Internal.EnumLiteMap<ReplicaStateProto>
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static com.google.protobuf.Internal.EnumLiteMap<ReplicaState>
+    private static com.google.protobuf.Internal.EnumLiteMap<ReplicaStateProto>
         internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap<ReplicaState>() {
-            public ReplicaState findValueByNumber(int number) {
-              return ReplicaState.valueOf(number);
+          new com.google.protobuf.Internal.EnumLiteMap<ReplicaStateProto>() {
+            public ReplicaStateProto findValueByNumber(int number) {
+              return ReplicaStateProto.valueOf(number);
             }
           };
     
@@ -62,11 +62,11 @@ public final class HdfsProtos {
       return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.getDescriptor().getEnumTypes().get(0);
     }
     
-    private static final ReplicaState[] VALUES = {
+    private static final ReplicaStateProto[] VALUES = {
       FINALIZED, RBW, RWR, RUR, TEMPORARY, 
     };
     
-    public static ReplicaState valueOf(
+    public static ReplicaStateProto valueOf(
         com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
@@ -78,12 +78,12 @@ public final class HdfsProtos {
     private final int index;
     private final int value;
     
-    private ReplicaState(int index, int value) {
+    private ReplicaStateProto(int index, int value) {
       this.index = index;
       this.value = value;
     }
     
-    // @@protoc_insertion_point(enum_scope:ReplicaState)
+    // @@protoc_insertion_point(enum_scope:ReplicaStateProto)
   }
   
   public interface ExtendedBlockProtoOrBuilder
@@ -14903,15 +14903,10 @@ public final class HdfsProtos {
     org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto getBlock();
     org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProtoOrBuilder getBlockOrBuilder();
     
-    // repeated .DatanodeIDProto datanodeIDs = 2;
-    java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> 
-        getDatanodeIDsList();
-    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDatanodeIDs(int index);
+    // repeated string datanodeIDs = 2;
+    java.util.List<String> getDatanodeIDsList();
     int getDatanodeIDsCount();
-    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> 
-        getDatanodeIDsOrBuilderList();
-    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getDatanodeIDsOrBuilder(
-        int index);
+    String getDatanodeIDs(int index);
   }
   public static final class BlockWithLocationsProto extends
       com.google.protobuf.GeneratedMessage
@@ -14955,30 +14950,23 @@ public final class HdfsProtos {
       return block_;
     }
     
-    // repeated .DatanodeIDProto datanodeIDs = 2;
+    // repeated string datanodeIDs = 2;
     public static final int DATANODEIDS_FIELD_NUMBER = 2;
-    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> datanodeIDs_;
-    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> getDatanodeIDsList() {
-      return datanodeIDs_;
-    }
-    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> 
-        getDatanodeIDsOrBuilderList() {
+    private com.google.protobuf.LazyStringList datanodeIDs_;
+    public java.util.List<String>
+        getDatanodeIDsList() {
       return datanodeIDs_;
     }
     public int getDatanodeIDsCount() {
       return datanodeIDs_.size();
     }
-    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDatanodeIDs(int index) {
-      return datanodeIDs_.get(index);
-    }
-    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getDatanodeIDsOrBuilder(
-        int index) {
+    public String getDatanodeIDs(int index) {
       return datanodeIDs_.get(index);
     }
     
     private void initFields() {
       block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance();
-      datanodeIDs_ = java.util.Collections.emptyList();
+      datanodeIDs_ = com.google.protobuf.LazyStringArrayList.EMPTY;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -14993,12 +14981,6 @@ public final class HdfsProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      for (int i = 0; i < getDatanodeIDsCount(); i++) {
-        if (!getDatanodeIDs(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -15010,7 +14992,7 @@ public final class HdfsProtos {
         output.writeMessage(1, block_);
       }
       for (int i = 0; i < datanodeIDs_.size(); i++) {
-        output.writeMessage(2, datanodeIDs_.get(i));
+        output.writeBytes(2, datanodeIDs_.getByteString(i));
       }
       getUnknownFields().writeTo(output);
     }
@@ -15025,9 +15007,14 @@ public final class HdfsProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, block_);
       }
-      for (int i = 0; i < datanodeIDs_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, datanodeIDs_.get(i));
+      {
+        int dataSize = 0;
+        for (int i = 0; i < datanodeIDs_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(datanodeIDs_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getDatanodeIDsList().size();
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -15185,7 +15172,6 @@ public final class HdfsProtos {
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getBlockFieldBuilder();
-          getDatanodeIDsFieldBuilder();
         }
       }
       private static Builder create() {
@@ -15200,12 +15186,8 @@ public final class HdfsProtos {
           blockBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        if (datanodeIDsBuilder_ == null) {
-          datanodeIDs_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
-        } else {
-          datanodeIDsBuilder_.clear();
-        }
+        datanodeIDs_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
       
@@ -15252,15 +15234,12 @@ public final class HdfsProtos {
         } else {
           result.block_ = blockBuilder_.build();
         }
-        if (datanodeIDsBuilder_ == null) {
-          if (((bitField0_ & 0x00000002) == 0x00000002)) {
-            datanodeIDs_ = java.util.Collections.unmodifiableList(datanodeIDs_);
-            bitField0_ = (bitField0_ & ~0x00000002);
-          }
-          result.datanodeIDs_ = datanodeIDs_;
-        } else {
-          result.datanodeIDs_ = datanodeIDsBuilder_.build();
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          datanodeIDs_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              datanodeIDs_);
+          bitField0_ = (bitField0_ & ~0x00000002);
         }
+        result.datanodeIDs_ = datanodeIDs_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -15280,31 +15259,15 @@ public final class HdfsProtos {
         if (other.hasBlock()) {
           mergeBlock(other.getBlock());
         }
-        if (datanodeIDsBuilder_ == null) {
-          if (!other.datanodeIDs_.isEmpty()) {
-            if (datanodeIDs_.isEmpty()) {
-              datanodeIDs_ = other.datanodeIDs_;
-              bitField0_ = (bitField0_ & ~0x00000002);
-            } else {
-              ensureDatanodeIDsIsMutable();
-              datanodeIDs_.addAll(other.datanodeIDs_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.datanodeIDs_.isEmpty()) {
-            if (datanodeIDsBuilder_.isEmpty()) {
-              datanodeIDsBuilder_.dispose();
-              datanodeIDsBuilder_ = null;
-              datanodeIDs_ = other.datanodeIDs_;
-              bitField0_ = (bitField0_ & ~0x00000002);
-              datanodeIDsBuilder_ = 
-                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
-                   getDatanodeIDsFieldBuilder() : null;
-            } else {
-              datanodeIDsBuilder_.addAllMessages(other.datanodeIDs_);
-            }
+        if (!other.datanodeIDs_.isEmpty()) {
+          if (datanodeIDs_.isEmpty()) {
+            datanodeIDs_ = other.datanodeIDs_;
+            bitField0_ = (bitField0_ & ~0x00000002);
+          } else {
+            ensureDatanodeIDsIsMutable();
+            datanodeIDs_.addAll(other.datanodeIDs_);
           }
+          onChanged();
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
@@ -15319,12 +15282,6 @@ public final class HdfsProtos {
           
           return false;
         }
-        for (int i = 0; i < getDatanodeIDsCount(); i++) {
-          if (!getDatanodeIDs(i).isInitialized()) {
-            
-            return false;
-          }
-        }
         return true;
       }
       
@@ -15361,9 +15318,8 @@ public final class HdfsProtos {
               break;
             }
             case 18: {
-              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.newBuilder();
-              input.readMessage(subBuilder, extensionRegistry);
-              addDatanodeIDs(subBuilder.buildPartial());
+              ensureDatanodeIDsIsMutable();
+              datanodeIDs_.add(input.readBytes());
               break;
             }
           }
@@ -15462,190 +15418,60 @@ public final class HdfsProtos {
         return blockBuilder_;
       }
       
-      // repeated .DatanodeIDProto datanodeIDs = 2;
-      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> datanodeIDs_ =
-        java.util.Collections.emptyList();
+      // repeated string datanodeIDs = 2;
+      private com.google.protobuf.LazyStringList datanodeIDs_ = com.google.protobuf.LazyStringArrayList.EMPTY;
       private void ensureDatanodeIDsIsMutable() {
         if (!((bitField0_ & 0x00000002) == 0x00000002)) {
-          datanodeIDs_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto>(datanodeIDs_);
+          datanodeIDs_ = new com.google.protobuf.LazyStringArrayList(datanodeIDs_);
           bitField0_ |= 0x00000002;
          }
       }
-      
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> datanodeIDsBuilder_;
-      
-      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> getDatanodeIDsList() {
-        if (datanodeIDsBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(datanodeIDs_);
-        } else {
-          return datanodeIDsBuilder_.getMessageList();
-        }
+      public java.util.List<String>
+          getDatanodeIDsList() {
+        return java.util.Collections.unmodifiableList(datanodeIDs_);
       }
       public int getDatanodeIDsCount() {
-        if (datanodeIDsBuilder_ == null) {
-          return datanodeIDs_.size();
-        } else {
-          return datanodeIDsBuilder_.getCount();
-        }
+        return datanodeIDs_.size();
       }
-      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDatanodeIDs(int index) {
-        if (datanodeIDsBuilder_ == null) {
-          return datanodeIDs_.get(index);
-        } else {
-          return datanodeIDsBuilder_.getMessage(index);
-        }
+      public String getDatanodeIDs(int index) {
+        return datanodeIDs_.get(index);
       }
       public Builder setDatanodeIDs(
-          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
-        if (datanodeIDsBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureDatanodeIDsIsMutable();
-          datanodeIDs_.set(index, value);
-          onChanged();
-        } else {
-          datanodeIDsBuilder_.setMessage(index, value);
-        }
-        return this;
-      }
-      public Builder setDatanodeIDs(
-          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder builderForValue) {
-        if (datanodeIDsBuilder_ == null) {
-          ensureDatanodeIDsIsMutable();
-          datanodeIDs_.set(index, builderForValue.build());
-          onChanged();
-        } else {
-          datanodeIDsBuilder_.setMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      public Builder addDatanodeIDs(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
-        if (datanodeIDsBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureDatanodeIDsIsMutable();
-          datanodeIDs_.add(value);
-          onChanged();
-        } else {
-          datanodeIDsBuilder_.addMessage(value);
-        }
-        return this;
-      }
-      public Builder addDatanodeIDs(
-          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
-        if (datanodeIDsBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureDatanodeIDsIsMutable();
-          datanodeIDs_.add(index, value);
-          onChanged();
-        } else {
-          datanodeIDsBuilder_.addMessage(index, value);
-        }
-        return this;
-      }
-      public Builder addDatanodeIDs(
-          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder builderForValue) {
-        if (datanodeIDsBuilder_ == null) {
-          ensureDatanodeIDsIsMutable();
-          datanodeIDs_.add(builderForValue.build());
-          onChanged();
-        } else {
-          datanodeIDsBuilder_.addMessage(builderForValue.build());
-        }
+          int index, String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureDatanodeIDsIsMutable();
+        datanodeIDs_.set(index, value);
+        onChanged();
         return this;
       }
-      public Builder addDatanodeIDs(
-          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder builderForValue) {
-        if (datanodeIDsBuilder_ == null) {
-          ensureDatanodeIDsIsMutable();
-          datanodeIDs_.add(index, builderForValue.build());
-          onChanged();
-        } else {
-          datanodeIDsBuilder_.addMessage(index, builderForValue.build());
-        }
+      public Builder addDatanodeIDs(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureDatanodeIDsIsMutable();
+        datanodeIDs_.add(value);
+        onChanged();
         return this;
       }
       public Builder addAllDatanodeIDs(
-          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto> values) {
-        if (datanodeIDsBuilder_ == null) {
-          ensureDatanodeIDsIsMutable();
-          super.addAll(values, datanodeIDs_);
-          onChanged();
-        } else {
-          datanodeIDsBuilder_.addAllMessages(values);
-        }
+          java.lang.Iterable<String> values) {
+        ensureDatanodeIDsIsMutable();
+        super.addAll(values, datanodeIDs_);
+        onChanged();
         return this;
       }
       public Builder clearDatanodeIDs() {
-        if (datanodeIDsBuilder_ == null) {
-          datanodeIDs_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
-          onChanged();
-        } else {
-          datanodeIDsBuilder_.clear();
-        }
-        return this;
-      }
-      public Builder removeDatanodeIDs(int index) {
-        if (datanodeIDsBuilder_ == null) {
-          ensureDatanodeIDsIsMutable();
-          datanodeIDs_.remove(index);
-          onChanged();
-        } else {
-          datanodeIDsBuilder_.remove(index);
-        }
+        datanodeIDs_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        onChanged();
         return this;
       }
-      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder getDatanodeIDsBuilder(
-          int index) {
-        return getDatanodeIDsFieldBuilder().getBuilder(index);
-      }
-      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getDatanodeIDsOrBuilder(
-          int index) {
-        if (datanodeIDsBuilder_ == null) {
-          return datanodeIDs_.get(index);  } else {
-          return datanodeIDsBuilder_.getMessageOrBuilder(index);
-        }
-      }
-      public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> 
-           getDatanodeIDsOrBuilderList() {
-        if (datanodeIDsBuilder_ != null) {
-          return datanodeIDsBuilder_.getMessageOrBuilderList();
-        } else {
-          return java.util.Collections.unmodifiableList(datanodeIDs_);
-        }
-      }
-      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder addDatanodeIDsBuilder() {
-        return getDatanodeIDsFieldBuilder().addBuilder(
-            org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance());
-      }
-      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder addDatanodeIDsBuilder(
-          int index) {
-        return getDatanodeIDsFieldBuilder().addBuilder(
-            index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance());
-      }
-      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder> 
-           getDatanodeIDsBuilderList() {
-        return getDatanodeIDsFieldBuilder().getBuilderList();
-      }
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> 
-          getDatanodeIDsFieldBuilder() {
-        if (datanodeIDsBuilder_ == null) {
-          datanodeIDsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
-              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder>(
-                  datanodeIDs_,
-                  ((bitField0_ & 0x00000002) == 0x00000002),
-                  getParentForChildren(),
-                  isClean());
-          datanodeIDs_ = null;
-        }
-        return datanodeIDsBuilder_;
+      void addDatanodeIDs(com.google.protobuf.ByteString value) {
+        ensureDatanodeIDsIsMutable();
+        datanodeIDs_.add(value);
+        onChanged();
       }
       
       // @@protoc_insertion_point(builder_scope:BlockWithLocationsProto)
@@ -20348,29 +20174,28 @@ public final class HdfsProtos {
       "ature\030\001 \002(\0132\031.CheckpointSignatureProto\022\031" +
       "\n\021needToReturnImage\030\002 \002(\010\"A\n\nBlockProto\022" +
       "\017\n\007blockId\030\001 \002(\004\022\020\n\010genStamp\030\002 \002(\004\022\020\n\010nu" +
-      "mBytes\030\003 \001(\004\"\\\n\027BlockWithLocationsProto\022",
-      "\032\n\005block\030\001 \002(\0132\013.BlockProto\022%\n\013datanodeI" +
-      "Ds\030\002 \003(\0132\020.DatanodeIDProto\"D\n\030BlocksWith" +
-      "LocationsProto\022(\n\006blocks\030\001 \003(\0132\030.BlockWi" +
-      "thLocationsProto\"8\n\022RemoteEditLogProto\022\021" +
-      "\n\tstartTxId\030\001 \002(\004\022\017\n\007endTxId\030\002 \002(\004\"?\n\032Re" +
-      "moteEditLogManifestProto\022!\n\004logs\030\001 \003(\0132\023" +
-      ".RemoteEditLogProto\"\203\001\n\022NamespaceInfoPro" +
-      "to\022\024\n\014buildVersion\030\001 \002(\t\022\032\n\022distUpgradeV" +
-      "ersion\030\002 \002(\r\022\023\n\013blockPoolID\030\003 \002(\t\022&\n\013sto" +
-      "rageInfo\030\004 \002(\0132\021.StorageInfoProto\"D\n\rBlo",
-      "ckKeyProto\022\r\n\005keyId\030\001 \002(\r\022\022\n\nexpiryDate\030" +
-      "\002 \002(\004\022\020\n\010keyBytes\030\003 \002(\014\"\254\001\n\026ExportedBloc" +
-      "kKeysProto\022\033\n\023isBlockTokenEnabled\030\001 \002(\010\022" +
-      "\031\n\021keyUpdateInterval\030\002 \002(\004\022\025\n\rtokenLifeT" +
-      "ime\030\003 \002(\004\022\"\n\ncurrentKey\030\004 \002(\0132\016.BlockKey" +
-      "Proto\022\037\n\007allKeys\030\005 \003(\0132\016.BlockKeyProto\"N" +
-      "\n\024RecoveringBlockProto\022\023\n\013newGenStamp\030\001 " +
-      "\002(\004\022!\n\005block\030\002 \002(\0132\022.LocatedBlockProto*G" +
-      "\n\014ReplicaState\022\r\n\tFINALIZED\020\000\022\007\n\003RBW\020\001\022\007" +
-      "\n\003RWR\020\002\022\007\n\003RUR\020\003\022\r\n\tTEMPORARY\020\004B6\n%org.a",
-      "pache.hadoop.hdfs.protocol.protoB\nHdfsPr" +
-      "otos\240\001\001"
+      "mBytes\030\003 \001(\004\"J\n\027BlockWithLocationsProto\022",
+      "\032\n\005block\030\001 \002(\0132\013.BlockProto\022\023\n\013datanodeI" +
+      "Ds\030\002 \003(\t\"D\n\030BlocksWithLocationsProto\022(\n\006" +
+      "blocks\030\001 \003(\0132\030.BlockWithLocationsProto\"8" +
+      "\n\022RemoteEditLogProto\022\021\n\tstartTxId\030\001 \002(\004\022" +
+      "\017\n\007endTxId\030\002 \002(\004\"?\n\032RemoteEditLogManifes" +
+      "tProto\022!\n\004logs\030\001 \003(\0132\023.RemoteEditLogProt" +
+      "o\"\203\001\n\022NamespaceInfoProto\022\024\n\014buildVersion" +
+      "\030\001 \002(\t\022\032\n\022distUpgradeVersion\030\002 \002(\r\022\023\n\013bl" +
+      "ockPoolID\030\003 \002(\t\022&\n\013storageInfo\030\004 \002(\0132\021.S" +
+      "torageInfoProto\"D\n\rBlockKeyProto\022\r\n\005keyI",
+      "d\030\001 \002(\r\022\022\n\nexpiryDate\030\002 \002(\004\022\020\n\010keyBytes\030" +
+      "\003 \002(\014\"\254\001\n\026ExportedBlockKeysProto\022\033\n\023isBl" +
+      "ockTokenEnabled\030\001 \002(\010\022\031\n\021keyUpdateInterv" +
+      "al\030\002 \002(\004\022\025\n\rtokenLifeTime\030\003 \002(\004\022\"\n\ncurre" +
+      "ntKey\030\004 \002(\0132\016.BlockKeyProto\022\037\n\007allKeys\030\005" +
+      " \003(\0132\016.BlockKeyProto\"N\n\024RecoveringBlockP" +
+      "roto\022\023\n\013newGenStamp\030\001 \002(\004\022!\n\005block\030\002 \002(\013" +
+      "2\022.LocatedBlockProto*L\n\021ReplicaStateProt" +
+      "o\022\r\n\tFINALIZED\020\000\022\007\n\003RBW\020\001\022\007\n\003RWR\020\002\022\007\n\003RU" +
+      "R\020\003\022\r\n\tTEMPORARY\020\004B6\n%org.apache.hadoop.",
+      "hdfs.protocol.protoB\nHdfsProtos\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

+ 30 - 29
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/InterDatanodeProtocolProtos.java

@@ -484,9 +484,9 @@ public final class InterDatanodeProtocolProtos {
   public interface InitReplicaRecoveryResponseProtoOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
     
-    // required .ReplicaState state = 1;
+    // required .ReplicaStateProto state = 1;
     boolean hasState();
-    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState getState();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto getState();
     
     // required .BlockProto block = 2;
     boolean hasBlock();
@@ -522,13 +522,13 @@ public final class InterDatanodeProtocolProtos {
     }
     
     private int bitField0_;
-    // required .ReplicaState state = 1;
+    // required .ReplicaStateProto state = 1;
     public static final int STATE_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState state_;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto state_;
     public boolean hasState() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
-    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState getState() {
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto getState() {
       return state_;
     }
     
@@ -546,7 +546,7 @@ public final class InterDatanodeProtocolProtos {
     }
     
     private void initFields() {
-      state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState.FINALIZED;
+      state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto.FINALIZED;
       block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
@@ -763,7 +763,7 @@ public final class InterDatanodeProtocolProtos {
       
       public Builder clear() {
         super.clear();
-        state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState.FINALIZED;
+        state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto.FINALIZED;
         bitField0_ = (bitField0_ & ~0x00000001);
         if (blockBuilder_ == null) {
           block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto.getDefaultInstance();
@@ -888,7 +888,7 @@ public final class InterDatanodeProtocolProtos {
             }
             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);
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto value = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto.valueOf(rawValue);
               if (value == null) {
                 unknownFields.mergeVarintField(1, rawValue);
               } else {
@@ -912,15 +912,15 @@ public final class InterDatanodeProtocolProtos {
       
       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;
+      // required .ReplicaStateProto state = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto.FINALIZED;
       public boolean hasState() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
-      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState getState() {
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto getState() {
         return state_;
       }
-      public Builder setState(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState value) {
+      public Builder setState(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto value) {
         if (value == null) {
           throw new NullPointerException();
         }
@@ -931,7 +931,7 @@ public final class InterDatanodeProtocolProtos {
       }
       public Builder clearState() {
         bitField0_ = (bitField0_ & ~0x00000001);
-        state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaState.FINALIZED;
+        state_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto.FINALIZED;
         onChanged();
         return this;
       }
@@ -2448,22 +2448,23 @@ public final class InterDatanodeProtocolProtos {
     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"
+      "block\030\001 \002(\0132\025.RecoveringBlockProto\"a\n In" +
+      "itReplicaRecoveryResponseProto\022!\n\005state\030" +
+      "\001 \002(\0162\022.ReplicaStateProto\022\032\n\005block\030\002 \002(\013" +
+      "2\013.BlockProto\"s\n&UpdateReplicaUnderRecov" +
+      "eryRequestProto\022\"\n\005block\030\001 \002(\0132\023.Extende" +
+      "dBlockProto\022\022\n\nrecoveryId\030\002 \002(\004\022\021\n\tnewLe" +
+      "ngth\030\003 \002(\004\"M\n\'UpdateReplicaUnderRecovery" +
+      "ResponseProto\022\"\n\005block\030\001 \002(\0132\023.ExtendedB",
+      "lockProto2\353\001\n\034InterDatanodeProtocolServi" +
+      "ce\022Z\n\023initReplicaRecovery\022 .InitReplicaR" +
+      "ecoveryRequestProto\032!.InitReplicaRecover" +
+      "yResponseProto\022o\n\032updateReplicaUnderReco" +
+      "very\022\'.UpdateReplicaUnderRecoveryRequest" +
+      "Proto\032(.UpdateReplicaUnderRecoveryRespon" +
+      "seProtoBJ\n%org.apache.hadoop.hdfs.protoc" +
+      "ol.protoB\033InterDatanodeProtocolProtos\210\001\001" +
+      "\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

파일 크기가 너무 크기때문에 변경 상태를 표시하지 않습니다.
+ 813 - 40
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/proto/NamenodeProtocolProtos.java


+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.token.TokenInfo;
+
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
+@TokenInfo(BlockTokenSelector.class)
+@ProtocolInfo(protocolName = 
+    "org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Private
+public interface ClientDatanodeProtocolPB extends
+    ClientDatanodeProtocolService.BlockingInterface, VersionedProtocol {
+  
+  /**
+   * This method is defined to get the protocol signature using 
+   * ProtocolSignatureWritable - suffix of 2 to the method name
+   * avoids conflict.
+   */
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException;
+}

+ 159 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java

@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.VersionedProtocol;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * Implementation for protobuf service that forwards requests
+ * received on {@link ClientDatanodeProtocolPB} to the
+ * {@link ClientDatanodeProtocol} server implementation.
+ */
+@InterfaceAudience.Private
+public class ClientDatanodeProtocolServerSideTranslatorPB implements
+    ClientDatanodeProtocolPB {
+  private final static RefreshNamenodesResponseProto REFRESH_NAMENODE_RESP =
+      RefreshNamenodesResponseProto.newBuilder().build();
+  private final static DeleteBlockPoolResponseProto DELETE_BLOCKPOOL_RESP =
+      DeleteBlockPoolResponseProto.newBuilder().build();
+  
+  private final ClientDatanodeProtocol impl;
+
+  public ClientDatanodeProtocolServerSideTranslatorPB(
+      ClientDatanodeProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public GetReplicaVisibleLengthResponseProto getReplicaVisibleLength(
+      RpcController unused, GetReplicaVisibleLengthRequestProto request)
+      throws ServiceException {
+    long len;
+    try {
+      len = impl.getReplicaVisibleLength(PBHelper.convert(request.getBlock()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetReplicaVisibleLengthResponseProto.newBuilder().setLength(len)
+        .build();
+  }
+
+  @Override
+  public RefreshNamenodesResponseProto refreshNamenode(
+      RpcController unused, RefreshNamenodesRequestProto request)
+      throws ServiceException {
+    try {
+      impl.refreshNamenodes();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return REFRESH_NAMENODE_RESP;
+  }
+
+  @Override
+  public DeleteBlockPoolResponseProto deleteBlockPool(RpcController unused,
+      DeleteBlockPoolRequestProto request) throws ServiceException {
+    try {
+      impl.deleteBlockPool(request.getBlockPool(), request.getForce());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return DELETE_BLOCKPOOL_RESP;
+  }
+
+  @Override
+  public GetBlockLocalPathInfoResponseProto getBlockLocalPathInfo(
+      RpcController unused, GetBlockLocalPathInfoRequestProto request)
+      throws ServiceException {
+    BlockLocalPathInfo resp;
+    try {
+      resp = impl.getBlockLocalPathInfo(PBHelper.convert(request.getBlock()), PBHelper.convert(request.getToken()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetBlockLocalPathInfoResponseProto.newBuilder()
+        .setBlock(PBHelper.convert(resp.getBlock()))
+        .setLocalPath(resp.getBlockPath()).setLocalMetaPath(resp.getMetaPath())
+        .build();
+  }
+
+  @Override
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    return RPC.getProtocolVersion(ClientDatanodeProtocolPB.class);
+  }
+
+  /**
+   * The client side will redirect getProtocolSignature to
+   * getProtocolSignature2.
+   * 
+   * However the RPC layer below on the Server side will call getProtocolVersion
+   * and possibly in the future getProtocolSignature. Hence we still implement
+   * it even though the end client will never call this method.
+   * 
+   * @see VersionedProtocol#getProtocolVersion
+   */
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and signature is
+     * that of {@link ClientDatanodeProtocol}
+     */
+    if (!protocol.equals(RPC.getProtocolName(ClientDatanodeProtocol.class))) {
+      throw new IOException("Namenode Serverside implements " +
+          RPC.getProtocolName(ClientDatanodeProtocol.class) +
+          ". The following requested protocol is unknown: " + protocol);
+    }
+
+    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
+        RPC.getProtocolVersion(ClientDatanodeProtocolPB.class),
+        ClientDatanodeProtocolPB.class);
+  }
+
+
+  @Override
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and signature is
+     * that of {@link ClientDatanodeProtocol}
+     */
+    return ProtocolSignatureWritable.convert(
+        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
+  }
+}

+ 136 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.token.Token;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link ClientDatanodeProtocol} interfaces to the RPC server implementing
+ * {@link ClientDatanodeProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ClientDatanodeProtocolTranslatorPB implements
+    ClientDatanodeProtocol, Closeable {
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final ClientDatanodeProtocolPB rpcProxy;
+  private final static RefreshNamenodesRequestProto REFRESH_NAMENODES = 
+      RefreshNamenodesRequestProto.newBuilder().build();
+
+
+  public ClientDatanodeProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
+      Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, ClientDatanodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    rpcProxy = RPC.getProxy(ClientDatanodeProtocolPB.class,
+        RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), nameNodeAddr,
+        conf);
+  }
+
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public long getProtocolVersion(String protocolName, long clientVersion)
+      throws IOException {
+    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
+        protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
+    GetReplicaVisibleLengthRequestProto req = GetReplicaVisibleLengthRequestProto
+        .newBuilder().setBlock(PBHelper.convert(b)).build();
+    try {
+      return rpcProxy.getReplicaVisibleLength(NULL_CONTROLLER, req).getLength();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void refreshNamenodes() throws IOException {
+    try {
+      rpcProxy.refreshNamenode(NULL_CONTROLLER, REFRESH_NAMENODES);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void deleteBlockPool(String bpid, boolean force) throws IOException {
+    DeleteBlockPoolRequestProto req = DeleteBlockPoolRequestProto.newBuilder()
+        .setBlockPool(bpid).setForce(force).build();
+    try {
+      rpcProxy.deleteBlockPool(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
+      Token<BlockTokenIdentifier> token) throws IOException {
+    GetBlockLocalPathInfoRequestProto req =
+        GetBlockLocalPathInfoRequestProto.newBuilder()
+        .setBlock(PBHelper.convert(block))
+        .setToken(PBHelper.convert(token)).build();
+    GetBlockLocalPathInfoResponseProto resp;
+    try {
+      resp = rpcProxy.getBlockLocalPathInfo(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    return new BlockLocalPathInfo(PBHelper.convert(resp.getBlock()),
+        resp.getLocalPath(), resp.getLocalMetaPath());
+  }
+}

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.KerberosInfo;
+
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY,
+    clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
+@ProtocolInfo(protocolName = 
+    "org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Private
+public interface InterDatanodeProtocolPB extends
+    InterDatanodeProtocolService.BlockingInterface, VersionedProtocol {
+
+  /**
+   * This method is defined to get the protocol signature using 
+   * the R23 protocol - hence we have added the suffix of 2 the method name
+   * to avoid conflict.
+   */
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException;
+}

+ 130 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java

@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.VersionedProtocol;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * Implementation for protobuf service that forwards requests
+ * received on {@link InterDatanodeProtocolPB} to the
+ * {@link InterDatanodeProtocol} server implementation.
+ */
+@InterfaceAudience.Private
+public class InterDatanodeProtocolServerSideTranslatorPB implements
+    InterDatanodeProtocolPB {
+  private final InterDatanodeProtocol impl;
+
+  public InterDatanodeProtocolServerSideTranslatorPB(InterDatanodeProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public InitReplicaRecoveryResponseProto initReplicaRecovery(
+      RpcController unused, InitReplicaRecoveryRequestProto request)
+      throws ServiceException {
+    RecoveringBlock b = PBHelper.convert(request.getBlock());
+    ReplicaRecoveryInfo r;
+    try {
+      r = impl.initReplicaRecovery(b);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return InitReplicaRecoveryResponseProto.newBuilder()
+        .setBlock(PBHelper.convert(r)).build();
+  }
+
+  @Override
+  public UpdateReplicaUnderRecoveryResponseProto updateReplicaUnderRecovery(
+      RpcController unused, UpdateReplicaUnderRecoveryRequestProto request)
+      throws ServiceException {
+    ExtendedBlock b;
+    try {
+      b = impl.updateReplicaUnderRecovery(PBHelper.convert(request.getBlock()),
+          request.getRecoveryId(), request.getNewLength());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
+        .setBlock(PBHelper.convert(b)).build();
+  }
+
+  /** @see VersionedProtocol#getProtocolVersion */
+  @Override
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    return RPC.getProtocolVersion(InterDatanodeProtocolPB.class);
+  }
+  
+  /**
+   * The client side will redirect getProtocolSignature to
+   * getProtocolSignature2.
+   * 
+   * However the RPC layer below on the Server side will call getProtocolVersion
+   * and possibly in the future getProtocolSignature. Hence we still implement
+   * it even though the end client will never call this method.
+   * 
+   * @see VersionedProtocol#getProtocolVersion
+   */
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and signature is
+     * that of {@link InterDatanodeProtocol}
+     */
+    if (!protocol.equals(RPC.getProtocolName(InterDatanodeProtocol.class))) {
+      throw new IOException("Namenode Serverside implements " +
+          RPC.getProtocolName(InterDatanodeProtocol.class) +
+          ". The following requested protocol is unknown: " + protocol);
+    }
+
+    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
+        RPC.getProtocolVersion(InterDatanodeProtocolPB.class),
+        InterDatanodeProtocolPB.class);
+  }
+
+
+  @Override
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and signature is
+     * that of {@link InterDatanodeProtocol}
+     */
+    return ProtocolSignatureWritable.convert(
+        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
+  }
+}

+ 114 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java

@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link InterDatanodeProtocol} interfaces to the RPC server implementing
+ * {@link InterDatanodeProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class InterDatanodeProtocolTranslatorPB implements
+    InterDatanodeProtocol, Closeable {
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  final private InterDatanodeProtocolPB rpcProxy;
+
+  public InterDatanodeProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
+      Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, InterDatanodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    rpcProxy = RPC.getProxy(InterDatanodeProtocolPB.class,
+        RPC.getProtocolVersion(InterDatanodeProtocolPB.class), nameNodeAddr,
+        conf);
+  }
+
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public long getProtocolVersion(String protocolName, long clientVersion)
+      throws IOException {
+    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
+        protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
+      throws IOException {
+    InitReplicaRecoveryRequestProto req = InitReplicaRecoveryRequestProto
+        .newBuilder().setBlock(PBHelper.convert(rBlock)).build();
+    InitReplicaRecoveryResponseProto resp;
+    try {
+      resp = rpcProxy.initReplicaRecovery(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    BlockProto b = resp.getBlock();
+    return new ReplicaRecoveryInfo(b.getBlockId(), b.getNumBytes(),
+        b.getGenStamp(), PBHelper.convert(resp.getState()));
+  }
+
+  @Override
+  public ExtendedBlock updateReplicaUnderRecovery(ExtendedBlock oldBlock,
+      long recoveryId, long newLength) throws IOException {
+    UpdateReplicaUnderRecoveryRequestProto req = 
+        UpdateReplicaUnderRecoveryRequestProto.newBuilder()
+        .setBlock(PBHelper.convert(oldBlock))
+        .setNewLength(newLength).setRecoveryId(recoveryId).build();
+    try {
+      return PBHelper.convert(rpcProxy.updateReplicaUnderRecovery(
+          NULL_CONTROLLER, req).getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+}

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.protocolPB;
 
 import java.io.IOException;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto;
@@ -37,6 +38,7 @@ import com.google.protobuf.ServiceException;
  * received on {@link JournalProtocolPB} to the 
  * {@link JournalProtocol} server implementation.
  */
+@InterfaceAudience.Private
 public class JournalProtocolServerSideTranslatorPB implements JournalProtocolPB {
   /** Server side implementation to delegate the requests to */
   private final JournalProtocol impl;
@@ -118,4 +120,4 @@ public class JournalProtocolServerSideTranslatorPB implements JournalProtocolPB
     return ProtocolSignatureWritable.convert(
         this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
   }
-}
+}

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java

@@ -53,7 +53,7 @@ public class JournalProtocolTranslatorPB implements JournalProtocol, Closeable {
       Configuration conf) throws IOException {
     RPC.setProtocolEngine(conf, JournalProtocolPB.class, ProtobufRpcEngine.class);
     rpcProxy = RPC.getProxy(JournalProtocolPB.class,
-        JournalProtocol.versionID, nameNodeAddr, conf);
+        RPC.getProtocolVersion(JournalProtocolPB.class), nameNodeAddr, conf);
   }
 
   @Override
@@ -64,7 +64,7 @@ public class JournalProtocolTranslatorPB implements JournalProtocol, Closeable {
   @Override
   public long getProtocolVersion(String protocolName, long clientVersion)
       throws IOException {
-    return 0;
+    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
   }
 
   @Override

+ 53 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.KerberosInfo;
+
+/**
+ * Protocol that a secondary NameNode uses to communicate with the NameNode.
+ * It's used to get part of the name node state
+ * 
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations required for security.
+ */
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
+    clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+@ProtocolInfo(protocolName = 
+    "org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Private
+public interface NamenodeProtocolPB extends
+    NamenodeProtocolService.BlockingInterface, VersionedProtocol {
+  /**
+   * This method is defined to get the protocol signature using 
+   * the R23 protocol - hence we have added the suffix of 2 the method name
+   * to avoid conflict.
+   */
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException;
+}

+ 253 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java

@@ -0,0 +1,253 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.VersionResponseProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * Implementation for protobuf service that forwards requests
+ * received on {@link NamenodeProtocolPB} to the
+ * {@link NamenodeProtocol} server implementation.
+ */
+public class NamenodeProtocolServerSideTranslatorPB implements
+    NamenodeProtocolPB {
+  private final NamenodeProtocol impl;
+
+  public NamenodeProtocolServerSideTranslatorPB(NamenodeProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public GetBlocksResponseProto getBlocks(RpcController unused,
+      GetBlocksRequestProto request) throws ServiceException {
+    DatanodeInfo dnInfo = new DatanodeInfo(PBHelper.convert(request
+        .getDatanode()));
+    BlocksWithLocations blocks;
+    try {
+      blocks = impl.getBlocks(dnInfo, request.getSize());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetBlocksResponseProto.newBuilder()
+        .setBlocks(PBHelper.convert(blocks)).build();
+  }
+
+  @Override
+  public GetBlockKeysResponseProto getBlockKeys(RpcController unused,
+      GetBlockKeysRequestProto request) throws ServiceException {
+    ExportedBlockKeys keys;
+    try {
+      keys = impl.getBlockKeys();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetBlockKeysResponseProto.newBuilder()
+        .setKeys(PBHelper.convert(keys)).build();
+  }
+
+  @Override
+  public GetTransactionIdResponseProto getTransationId(RpcController unused,
+      GetTransactionIdRequestProto request) throws ServiceException {
+    long txid;
+    try {
+      txid = impl.getTransactionID();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetTransactionIdResponseProto.newBuilder().setTxId(txid).build();
+  }
+
+  @Override
+  public RollEditLogResponseProto rollEditLog(RpcController unused,
+      RollEditLogRequestProto request) throws ServiceException {
+    CheckpointSignature signature;
+    try {
+      signature = impl.rollEditLog();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return RollEditLogResponseProto.newBuilder()
+        .setSignature(PBHelper.convert(signature)).build();
+  }
+
+  @Override
+  public ErrorReportResponseProto errorReport(RpcController unused,
+      ErrorReportRequestProto request) throws ServiceException {
+    try {
+      impl.errorReport(PBHelper.convert(request.getRegistration()),
+          request.getErrorCode(), request.getMsg());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return ErrorReportResponseProto.newBuilder().build();
+  }
+
+  @Override
+  public RegisterResponseProto register(RpcController unused,
+      RegisterRequestProto request) throws ServiceException {
+    NamenodeRegistration reg;
+    try {
+      reg = impl.register(PBHelper.convert(request.getRegistration()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return RegisterResponseProto.newBuilder()
+        .setRegistration(PBHelper.convert(reg)).build();
+  }
+
+  @Override
+  public StartCheckpointResponseProto startCheckpoint(RpcController unused,
+      StartCheckpointRequestProto request) throws ServiceException {
+    NamenodeCommand cmd;
+    try {
+      cmd = impl.startCheckpoint(PBHelper.convert(request.getRegistration()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return StartCheckpointResponseProto.newBuilder()
+        .setCommand(PBHelper.convert(cmd)).build();
+  }
+
+  @Override
+  public EndCheckpointResponseProto endCheckpoint(RpcController unused,
+      EndCheckpointRequestProto request) throws ServiceException {
+    try {
+      impl.endCheckpoint(PBHelper.convert(request.getRegistration()),
+          PBHelper.convert(request.getSignature()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return EndCheckpointResponseProto.newBuilder().build();
+  }
+
+  @Override
+  public GetEditLogManifestResponseProto getEditLogManifest(
+      RpcController unused, GetEditLogManifestRequestProto request)
+      throws ServiceException {
+    RemoteEditLogManifest manifest;
+    try {
+      manifest = impl.getEditLogManifest(request.getSinceTxId());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetEditLogManifestResponseProto.newBuilder()
+        .setManifest(PBHelper.convert(manifest)).build();
+  }
+  
+  @Override
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    return RPC.getProtocolVersion(NamenodeProtocolPB.class);
+  }
+
+  /**
+   * The client side will redirect getProtocolSignature to
+   * getProtocolSignature2.
+   * 
+   * However the RPC layer below on the Server side will call getProtocolVersion
+   * and possibly in the future getProtocolSignature. Hence we still implement
+   * it even though the end client will never call this method.
+   */
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and signature is
+     * that of {@link NamenodeProtocol}
+     */
+    if (!protocol.equals(RPC.getProtocolName(NamenodeProtocolPB.class))) {
+      throw new IOException("Namenode Serverside implements " +
+          RPC.getProtocolName(NamenodeProtocolPB.class) +
+          ". The following requested protocol is unknown: " + protocol);
+    }
+
+    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
+        RPC.getProtocolVersion(NamenodeProtocolPB.class),
+        NamenodeProtocolPB.class);
+  }
+
+
+  @Override
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and signature is
+     * that of {@link NamenodePBProtocol}
+     */
+    return ProtocolSignatureWritable.convert(
+        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public VersionResponseProto versionRequest(RpcController controller,
+      VersionRequestProto request) throws ServiceException {
+    NamespaceInfo info;
+    try {
+      info = impl.versionRequest();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VersionResponseProto.newBuilder()
+        .setInfo(convert(info)).build();
+  }
+
+  private NamespaceInfoProto convert(NamespaceInfo info) {
+    return NamespaceInfoProto.newBuilder()
+        .setBlockPoolID(info.getBlockPoolID())
+        .setBuildVersion(info.getBuildVersion())
+        .setDistUpgradeVersion(info.getDistributedUpgradeVersion())
+        .setStorageInfo(PBHelper.convert(info)).build();
+  }
+}

+ 270 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java

@@ -0,0 +1,270 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link NamenodeProtocol} interfaces to the RPC server implementing
+ * {@link NamenodeProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
+    Closeable {
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  
+  /*
+   * Protobuf requests with no parameters instantiated only once
+   */
+  private static final GetBlockKeysRequestProto GET_BLOCKKEYS = 
+      GetBlockKeysRequestProto.newBuilder().build();
+  private static final GetTransactionIdRequestProto GET_TRANSACTIONID = 
+      GetTransactionIdRequestProto.newBuilder().build();
+  private static final RollEditLogRequestProto ROLL_EDITLOG = 
+      RollEditLogRequestProto.newBuilder().build();
+  private static final VersionRequestProto VERSION_REQUEST = 
+      VersionRequestProto.newBuilder().build();
+
+  final private NamenodeProtocolPB rpcProxy;
+
+
+
+  private static NamenodeProtocolPB createNamenode(
+      InetSocketAddress nameNodeAddr, Configuration conf,
+      UserGroupInformation ugi) throws IOException {
+    return RPC.getProxy(NamenodeProtocolPB.class,
+        RPC.getProtocolVersion(NamenodeProtocolPB.class), nameNodeAddr, ugi,
+        conf, NetUtils.getSocketFactory(conf, NamenodeProtocolPB.class));
+  }
+
+  /** Create a {@link NameNode} proxy */
+  static NamenodeProtocolPB createNamenodeWithRetry(
+      NamenodeProtocolPB rpcNamenode) {
+    RetryPolicy createPolicy = RetryPolicies
+        .retryUpToMaximumCountWithFixedSleep(5,
+            HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
+    Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap = new HashMap<Class<? extends Exception>, RetryPolicy>();
+    remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
+        createPolicy);
+
+    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap = new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
+        .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
+            remoteExceptionToPolicyMap));
+    RetryPolicy methodPolicy = RetryPolicies.retryByException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+    Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
+
+    methodNameToPolicyMap.put("create", methodPolicy);
+
+    return (NamenodeProtocolPB) RetryProxy.create(NamenodeProtocolPB.class,
+        rpcNamenode, methodNameToPolicyMap);
+  }
+
+  public NamenodeProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
+      Configuration conf, UserGroupInformation ugi) throws IOException {
+    rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
+  }
+
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocolName,
+      long clientVersion, int clientMethodHash) throws IOException {
+    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
+        protocolName, clientVersion, clientMethodHash));
+  }
+
+  @Override
+  public long getProtocolVersion(String protocolName, long clientVersion)
+      throws IOException {
+    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
+  }
+
+  @Override
+  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
+      throws IOException {
+    GetBlocksRequestProto req = GetBlocksRequestProto.newBuilder()
+        .setDatanode(PBHelper.convert((DatanodeID)datanode)).setSize(size)
+        .build();
+    try {
+      return PBHelper.convert(rpcProxy.getBlocks(NULL_CONTROLLER, req)
+          .getBlocks());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public ExportedBlockKeys getBlockKeys() throws IOException {
+    try {
+      return PBHelper.convert(rpcProxy.getBlockKeys(NULL_CONTROLLER,
+          GET_BLOCKKEYS).getKeys());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long getTransactionID() throws IOException {
+    try {
+      return rpcProxy.getTransationId(NULL_CONTROLLER, GET_TRANSACTIONID)
+          .getTxId();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public CheckpointSignature rollEditLog() throws IOException {
+    try {
+      return PBHelper.convert(rpcProxy.rollEditLog(NULL_CONTROLLER,
+          ROLL_EDITLOG).getSignature());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public NamespaceInfo versionRequest() throws IOException {
+    try {
+      return PBHelper.convert(rpcProxy.versionRequest(NULL_CONTROLLER,
+          VERSION_REQUEST).getInfo());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void errorReport(NamenodeRegistration registration, int errorCode,
+      String msg) throws IOException {
+    ErrorReportRequestProto req = ErrorReportRequestProto.newBuilder()
+        .setErrorCode(errorCode).setMsg(msg)
+        .setRegistration(PBHelper.convert(registration)).build();
+    try {
+      rpcProxy.errorReport(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public NamenodeRegistration register(NamenodeRegistration registration)
+      throws IOException {
+    RegisterRequestProto req = RegisterRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration)).build();
+    try {
+      return PBHelper.convert(rpcProxy.register(NULL_CONTROLLER, req)
+          .getRegistration());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
+      throws IOException {
+    StartCheckpointRequestProto req = StartCheckpointRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration)).build();
+    NamenodeCommandProto cmd;
+    try {
+      cmd = rpcProxy.startCheckpoint(NULL_CONTROLLER, req).getCommand();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    return PBHelper.convert(cmd);
+  }
+
+  @Override
+  public void endCheckpoint(NamenodeRegistration registration,
+      CheckpointSignature sig) throws IOException {
+    EndCheckpointRequestProto req = EndCheckpointRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration))
+        .setSignature(PBHelper.convert(sig)).build();
+    try {
+      rpcProxy.endCheckpoint(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
+      throws IOException {
+    GetEditLogManifestRequestProto req = GetEditLogManifestRequestProto
+        .newBuilder().setSinceTxId(sinceTxId).build();
+    try {
+      return PBHelper.convert(rpcProxy.getEditLogManifest(NULL_CONTROLLER, req)
+          .getManifest());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+}

+ 344 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -17,28 +17,72 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
+import org.apache.hadoop.hdfs.security.token.block.BlockKey;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
 
 import com.google.protobuf.ByteString;
 
 /**
- * Utilities for converting protobuf classes to and from 
- * implementation classes.
+ * Utilities for converting protobuf classes to and from implementation classes.
  */
 class PBHelper {
   private PBHelper() {
     /** Hidden constructor */
   }
-  
+
   public static ByteString getByteString(byte[] bytes) {
     return ByteString.copyFrom(bytes);
   }
-  
+
   public static NamenodeRole convert(NamenodeRoleProto role) {
     switch (role) {
     case NAMENODE:
@@ -50,7 +94,7 @@ class PBHelper {
     }
     return null;
   }
-  
+
   public static NamenodeRoleProto convert(NamenodeRole role) {
     switch (role) {
     case NAMENODE:
@@ -62,31 +106,315 @@ class PBHelper {
     }
     return null;
   }
-  
+
   public static StorageInfoProto convert(StorageInfo info) {
     return StorageInfoProto.newBuilder().setClusterID(info.getClusterID())
-        .setCTime(info.getCTime())
-        .setLayoutVersion(info.getLayoutVersion())
-        .setNamespceID(info.getNamespaceID())
-        .build();
+        .setCTime(info.getCTime()).setLayoutVersion(info.getLayoutVersion())
+        .setNamespceID(info.getNamespaceID()).build();
   }
-  
+
   public static StorageInfo convert(StorageInfoProto info) {
     return new StorageInfo(info.getLayoutVersion(), info.getNamespceID(),
         info.getClusterID(), info.getCTime());
   }
-  
-  
+
   public static NamenodeRegistrationProto convert(NamenodeRegistration reg) {
     return NamenodeRegistrationProto.newBuilder()
-        .setHttpAddress(reg.getHttpAddress())
-        .setRole(convert(reg.getRole()))
+        .setHttpAddress(reg.getHttpAddress()).setRole(convert(reg.getRole()))
         .setRpcAddress(reg.getAddress())
         .setStorageInfo(convert((StorageInfo) reg)).build();
   }
-  
+
   public static NamenodeRegistration convert(NamenodeRegistrationProto reg) {
     return new NamenodeRegistration(reg.getRpcAddress(), reg.getHttpAddress(),
         convert(reg.getStorageInfo()), convert(reg.getRole()));
   }
+
+  public static DatanodeID convert(DatanodeIDProto dn) {
+    return new DatanodeID(dn.getName(), dn.getStorageID(), dn.getInfoPort(),
+        dn.getIpcPort());
+  }
+
+  public static DatanodeIDProto convert(DatanodeID dn) {
+    return DatanodeIDProto.newBuilder().setName(dn.getName())
+        .setInfoPort(dn.getInfoPort()).setIpcPort(dn.getIpcPort())
+        .setStorageID(dn.getStorageID()).build();
+  }
+
+  public static BlockProto convert(Block b) {
+    return BlockProto.newBuilder().setBlockId(b.getBlockId())
+        .setGenStamp(b.getGenerationStamp()).setNumBytes(b.getNumBytes())
+        .build();
+  }
+
+  public static Block convert(BlockProto b) {
+    return new Block(b.getBlockId(), b.getGenStamp(), b.getNumBytes());
+  }
+
+  public static BlockWithLocationsProto convert(BlockWithLocations blk) {
+    return BlockWithLocationsProto.newBuilder()
+        .setBlock(convert(blk.getBlock()))
+        .addAllDatanodeIDs(Arrays.asList(blk.getDatanodes())).build();
+  }
+
+  public static BlockWithLocations convert(BlockWithLocationsProto b) {
+    return new BlockWithLocations(convert(b.getBlock()), b.getDatanodeIDsList()
+        .toArray(new String[0]));
+  }
+
+  public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
+    BlocksWithLocationsProto.Builder builder = BlocksWithLocationsProto
+        .newBuilder();
+    for (BlockWithLocations b : blks.getBlocks()) {
+      builder.addBlocks(convert(b));
+    }
+    return builder.build();
+  }
+
+  public static BlocksWithLocations convert(BlocksWithLocationsProto blocks) {
+    List<BlockWithLocationsProto> b = blocks.getBlocksList();
+    BlockWithLocations[] ret = new BlockWithLocations[b.size()];
+    int i = 0;
+    for (BlockWithLocationsProto entry : b) {
+      ret[i++] = convert(entry);
+    }
+    return new BlocksWithLocations(ret);
+  }
+
+  public static BlockKeyProto convert(BlockKey key) {
+    byte[] encodedKey = key.getEncodedKey();
+    ByteString keyBytes = ByteString.copyFrom(encodedKey == null ? new byte[0]
+        : encodedKey);
+    return BlockKeyProto.newBuilder().setKeyId(key.getKeyId())
+        .setKeyBytes(keyBytes).setExpiryDate(key.getExpiryDate()).build();
+  }
+
+  public static BlockKey convert(BlockKeyProto k) {
+    return new BlockKey(k.getKeyId(), k.getExpiryDate(), k.getKeyBytes()
+        .toByteArray());
+  }
+
+  public static ExportedBlockKeysProto convert(ExportedBlockKeys keys) {
+    ExportedBlockKeysProto.Builder builder = ExportedBlockKeysProto
+        .newBuilder();
+    builder.setIsBlockTokenEnabled(keys.isBlockTokenEnabled())
+        .setKeyUpdateInterval(keys.getKeyUpdateInterval())
+        .setTokenLifeTime(keys.getTokenLifetime())
+        .setCurrentKey(convert(keys.getCurrentKey()));
+    for (BlockKey k : keys.getAllKeys()) {
+      builder.addAllKeys(convert(k));
+    }
+    return builder.build();
+  }
+
+  public static ExportedBlockKeys convert(ExportedBlockKeysProto keys) {
+    return new ExportedBlockKeys(keys.getIsBlockTokenEnabled(),
+        keys.getKeyUpdateInterval(), keys.getTokenLifeTime(),
+        convert(keys.getCurrentKey()), convertBlockKeys(keys.getAllKeysList()));
+  }
+
+  public static CheckpointSignatureProto convert(CheckpointSignature s) {
+    return CheckpointSignatureProto.newBuilder()
+        .setBlockPoolId(s.getBlockpoolID())
+        .setCurSegmentTxId(s.getCurSegmentTxId())
+        .setMostRecentCheckpointTxId(s.getMostRecentCheckpointTxId())
+        .setStorageInfo(PBHelper.convert((StorageInfo) s)).build();
+  }
+
+  public static CheckpointSignature convert(CheckpointSignatureProto s) {
+    return new CheckpointSignature(PBHelper.convert(s.getStorageInfo()),
+        s.getBlockPoolId(), s.getMostRecentCheckpointTxId(),
+        s.getCurSegmentTxId());
+  }
+
+  public static RemoteEditLogProto convert(RemoteEditLog log) {
+    return RemoteEditLogProto.newBuilder().setEndTxId(log.getEndTxId())
+        .setStartTxId(log.getStartTxId()).build();
+  }
+
+  public static RemoteEditLog convert(RemoteEditLogProto l) {
+    return new RemoteEditLog(l.getStartTxId(), l.getEndTxId());
+  }
+
+  public static RemoteEditLogManifestProto convert(
+      RemoteEditLogManifest manifest) {
+    RemoteEditLogManifestProto.Builder builder = RemoteEditLogManifestProto
+        .newBuilder();
+    for (RemoteEditLog log : manifest.getLogs()) {
+      builder.addLogs(convert(log));
+    }
+    return builder.build();
+  }
+
+  public static RemoteEditLogManifest convert(
+      RemoteEditLogManifestProto manifest) {
+    List<RemoteEditLog> logs = new ArrayList<RemoteEditLog>(manifest
+        .getLogsList().size());
+    for (RemoteEditLogProto l : manifest.getLogsList()) {
+      logs.add(convert(l));
+    }
+    return new RemoteEditLogManifest(logs);
+  }
+
+  public static CheckpointCommandProto convert(CheckpointCommand cmd) {
+    return CheckpointCommandProto.newBuilder()
+        .setSignature(convert(cmd.getSignature())).build();
+  }
+
+  public static NamenodeCommandProto convert(NamenodeCommand cmd) {
+    if (cmd instanceof CheckpointCommand) {
+      return NamenodeCommandProto.newBuilder().setAction(cmd.getAction())
+          .setType(NamenodeCommandProto.Type.NamenodeCommand)
+          .setCheckpointCmd(convert((CheckpointCommand) cmd)).build();
+    }
+    return NamenodeCommandProto.newBuilder().setAction(cmd.getAction()).build();
+  }
+
+  public static BlockKey[] convertBlockKeys(List<BlockKeyProto> list) {
+    BlockKey[] ret = new BlockKey[list.size()];
+    int i = 0;
+    for (BlockKeyProto k : list) {
+      ret[i++] = convert(k);
+    }
+    return ret;
+  }
+
+  public static NamespaceInfo convert(NamespaceInfoProto info) {
+    StorageInfoProto storage = info.getStorageInfo();
+    return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
+        info.getBlockPoolID(), storage.getCTime(), info.getDistUpgradeVersion());
+  }
+
+  public static NamenodeCommand convert(NamenodeCommandProto cmd) {
+    switch (cmd.getType()) {
+    case CheckPointCommand:
+      CheckpointCommandProto chkPt = cmd.getCheckpointCmd();
+      return new CheckpointCommand(PBHelper.convert(chkPt.getSignature()),
+          chkPt.getNeedToReturnImage());
+    default:
+      return new NamenodeCommand(cmd.getAction());
+    }
+  }
+
+  public static ExtendedBlockProto convert(ExtendedBlock b) {
+    return ExtendedBlockProto.newBuilder().setBlockId(b.getBlockId())
+        .setGenerationStamp(b.getGenerationStamp())
+        .setNumBytes(b.getNumBytes()).setPoolId(b.getBlockPoolId()).build();
+  }
+
+  public static ExtendedBlock convert(ExtendedBlockProto b) {
+    return new ExtendedBlock(b.getPoolId(), b.getBlockId(), b.getNumBytes(),
+        b.getGenerationStamp());
+  }
+
+  public static RecoveringBlockProto convert(RecoveringBlock b) {
+    LocatedBlockProto lb = PBHelper.convert((LocatedBlock)b);
+    return RecoveringBlockProto.newBuilder().setBlock(lb)
+        .setNewGenStamp(b.getNewGenerationStamp()).build();
+  }
+
+  public static RecoveringBlock convert(RecoveringBlockProto b) {
+    ExtendedBlock block = convert(b.getBlock().getB());
+    DatanodeInfo[] locs = convert(b.getBlock().getLocsList());
+    return new RecoveringBlock(block, locs, b.getNewGenStamp());
+  }
+
+  public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
+    DatanodeInfo[] info = new DatanodeInfo[list.size()];
+    for (int i = 0; i < info.length; i++) {
+      info[i] = convert(list.get(i));
+    }
+    return info;
+  }
+
+  public static DatanodeInfo convert(DatanodeInfoProto info) {
+    DatanodeIDProto dnId = info.getId();
+    return new DatanodeInfo(dnId.getName(), dnId.getStorageID(),
+        dnId.getInfoPort(), dnId.getIpcPort(), info.getCapacity(),
+        info.getDfsUsed(), info.getRemaining(), info.getBlockPoolUsed(),
+        info.getLastUpdate(), info.getXceiverCount(), info.getLocation(),
+        info.getHostName(), convert(info.getAdminState()));
+  }
+  
+  public static DatanodeInfoProto convert(DatanodeInfo info) {
+    return DatanodeInfoProto.newBuilder()
+        .setAdminState(PBHelper.convert(info.getAdminState()))
+        .setBlockPoolUsed(info.getBlockPoolUsed())
+        .setCapacity(info.getCapacity())
+        .setDfsUsed(info.getDfsUsed())
+        .setHostName(info.getHostName())
+        .setId(PBHelper.convert((DatanodeID)info))
+        .setLastUpdate(info.getLastUpdate())
+        .setLocation(info.getNetworkLocation())
+        .setRemaining(info.getRemaining())
+        .setXceiverCount(info.getXceiverCount())
+        .build();
+  }
+
+  public static AdminStates convert(AdminState adminState) {
+    switch(adminState) {
+    case DECOMMISSION_INPROGRESS:
+      return AdminStates.DECOMMISSION_INPROGRESS;
+    case DECOMMISSIONED:
+      return AdminStates.DECOMMISSIONED;
+    case NORMAL:
+    default:
+      return AdminStates.NORMAL;
+    }
+  }
+  
+  public static AdminState convert(AdminStates adminState) {
+    switch(adminState) {
+    case DECOMMISSION_INPROGRESS:
+      return AdminState.DECOMMISSION_INPROGRESS;
+    case DECOMMISSIONED:
+      return AdminState.DECOMMISSIONED;
+    case NORMAL:
+    default:
+      return AdminState.NORMAL;
+    }
+  }
+
+  public static LocatedBlockProto convert(LocatedBlock b) {
+    Builder builder = LocatedBlockProto.newBuilder();
+    DatanodeInfo[] locs = b.getLocations();
+    for(DatanodeInfo loc : locs) {
+      builder.addLocs(PBHelper.convert(loc));
+    }
+    return builder.setB(PBHelper.convert(b.getBlock()))
+        .setBlockToken(PBHelper.convert(b.getBlockToken()))
+        .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
+  }
+
+  public static BlockTokenIdentifierProto convert(
+      Token<BlockTokenIdentifier> token) {
+    ByteString tokenId = ByteString.copyFrom(token.getIdentifier());
+    ByteString password = ByteString.copyFrom(token.getPassword());
+    return BlockTokenIdentifierProto.newBuilder().setIdentifier(tokenId)
+        .setKind(token.getKind().toString()).setPassword(password)
+        .setService(token.getService().toString()).build();
+  }
+  
+  public static Token<BlockTokenIdentifier> convert(
+      BlockTokenIdentifierProto blockToken) {
+    return new Token<BlockTokenIdentifier>(blockToken.getIdentifier()
+        .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+        blockToken.getKind()), new Text(blockToken.getService()));
+  }
+
+  public static ReplicaState convert(ReplicaStateProto state) {
+    switch (state) {
+    case RBW:
+      return ReplicaState.RBW;
+    case RUR:
+      return ReplicaState.RUR;
+    case RWR:
+      return ReplicaState.RWR;
+    case TEMPORARY:
+      return ReplicaState.TEMPORARY;
+    case FINALIZED:
+    default:
+      return ReplicaState.FINALIZED;
+    }
+  }
 }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java

@@ -36,4 +36,8 @@ public class BlockKey extends DelegationKey {
   public BlockKey(int keyId, long expiryDate, SecretKey key) {
     super(keyId, expiryDate, key);
   }
+  
+  public BlockKey(int keyId, long expiryDate, byte[] encodedKey) {
+    super(keyId, expiryDate, encodedKey);
+  }
 }

+ 19 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Util.java

@@ -68,14 +68,28 @@ public final class Util {
   }
 
   /**
-   * Converts the passed File to a URI.
-   *
+   * Converts the passed File to a URI. This method trims the trailing slash if
+   * one is appended because the underlying file is in fact a directory that
+   * exists.
+   * 
    * @param f the file to convert
-   * @return the resulting URI 
-   * @throws IOException 
+   * @return the resulting URI
+   * @throws IOException
    */
   public static URI fileAsURI(File f) throws IOException {
-    return f.getCanonicalFile().toURI();
+    URI u = f.getCanonicalFile().toURI();
+    
+    // trim the trailing slash, if it's present
+    if (u.getPath().endsWith("/")) {
+      String uriAsString = u.toString();
+      try {
+        u = new URI(uriAsString.substring(0, uriAsString.length() - 1));
+      } catch (URISyntaxException e) {
+        throw new IOException(e);
+      }
+    }
+    
+    return u;
   }
 
   /**

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckableNameNodeResource.java

@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Implementers of this class represent a NN resource whose availability can be
+ * checked. A resource can be either "required" or "redundant". All required
+ * resources must be available for the NN to continue operating. The NN will
+ * continue to operate as long as *any* redundant resource is available.
+ */
+@InterfaceAudience.Private
+interface CheckableNameNodeResource {
+  
+  /**
+   * Is this resource currently available. 
+   * 
+   * @return true if and only if the resource in question is available.  
+   */
+  public boolean isResourceAvailable();
+  
+  /**
+   * Is this resource required.
+   * 
+   * @return true if and only if the resource in question is required for NN operation.
+   */
+  public boolean isRequired();
+
+}

+ 38 - 34
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -22,10 +22,10 @@ import java.net.URI;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.lang.reflect.Constructor;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -125,6 +125,13 @@ public class FSEditLog  {
 
   private NNStorage storage;
   private Configuration conf;
+  
+  private Collection<URI> editsDirs;
+  
+  /**
+   * The edit directories that are shared between primary and secondary.
+   */
+  private Collection<URI> sharedEditsDirs;
 
   private static class TransactionId {
     public long txid;
@@ -141,24 +148,22 @@ public class FSEditLog  {
     }
   };
 
-  final private Collection<URI> editsDirs;
-  
-  /**
-   * The edit directories that are shared between primary and secondary.
-   */
-  final private Collection<URI> sharedEditsDirs;
-
   /**
    * Construct FSEditLog with default configuration, taking editDirs from NNStorage
+   * 
    * @param storage Storage object used by namenode
    */
   @VisibleForTesting
-  FSEditLog(NNStorage storage) {
-    this(new Configuration(), storage, Collections.<URI>emptyList());
+  FSEditLog(NNStorage storage) throws IOException {
+    Configuration conf = new Configuration();
+    // Make sure the edits dirs are set in the provided configuration object.
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+        StringUtils.join(storage.getEditsDirectories(), ","));
+    init(conf, storage, FSNamesystem.getNamespaceEditsDirs(conf));
   }
 
   /**
-   * Constructor for FSEditLog. Add underlying journals are constructed, but 
+   * Constructor for FSEditLog. Underlying journals are constructed, but 
    * no streams are opened until open() is called.
    * 
    * @param conf The namenode configuration
@@ -166,27 +171,20 @@ public class FSEditLog  {
    * @param editsDirs List of journals to use
    */
   FSEditLog(Configuration conf, NNStorage storage, Collection<URI> editsDirs) {
-    this.conf = conf;
+    init(conf, storage, editsDirs);
+  }
+  
+  private void init(Configuration conf, NNStorage storage, Collection<URI> editsDirs) {
     isSyncRunning = false;
+    this.conf = conf;
     this.storage = storage;
     metrics = NameNode.getNameNodeMetrics();
     lastPrintTime = now();
-    
-    if (editsDirs.isEmpty()) { 
-      // if this is the case, no edit dirs have been explictly configured
-      // image dirs are to be used for edits too
-      try {
-        editsDirs = Lists.newArrayList(storage.getEditsDirectories());
-      } catch (IOException ioe) {
-        // cannot get list from storage, so the empty editsDirs 
-        // will be assigned. an error will be thrown on first use
-        // of the editlog, as no journals will exist
-      }
-      this.editsDirs = editsDirs;
-    } else {
-      this.editsDirs = Lists.newArrayList(editsDirs);
-    }
-    
+     
+    // If this list is empty, an error will be thrown on first use
+    // of the editlog, as no journals will exist
+    this.editsDirs = Lists.newArrayList(editsDirs);
+
     this.sharedEditsDirs = FSNamesystem.getSharedEditsDirs(conf);
   }
   
@@ -212,15 +210,21 @@ public class FSEditLog  {
   }
   
   private void initJournals(Collection<URI> dirs) {
-    this.journalSet = new JournalSet();
+    int minimumRedundantJournals = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY,
+        DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT);
+
+    journalSet = new JournalSet(minimumRedundantJournals);
     for (URI u : dirs) {
+      boolean required = FSNamesystem.getRequiredNamespaceEditsDirs(conf)
+          .contains(u);
       if (u.getScheme().equals(NNStorage.LOCAL_URI_SCHEME)) {
         StorageDirectory sd = storage.getStorageDirectory(u);
         if (sd != null) {
-          journalSet.add(new FileJournalManager(sd));
+          journalSet.add(new FileJournalManager(sd), required);
         }
       } else {
-        journalSet.add(createJournal(u));
+        journalSet.add(createJournal(u), required);
       }
     }
  
@@ -491,7 +495,7 @@ public class FSEditLog  {
             }
             editLogStream.setReadyToFlush();
           } catch (IOException e) {
-            LOG.fatal("Could not sync any journal to persistent storage. "
+            LOG.fatal("Could not sync enough journals to persistent storage. "
                 + "Unsynced transactions: " + (txid - synctxid),
                 new Exception());
             runtime.exit(1);
@@ -513,7 +517,7 @@ public class FSEditLog  {
         }
       } catch (IOException ex) {
         synchronized (this) {
-          LOG.fatal("Could not sync any journal to persistent storage. "
+          LOG.fatal("Could not sync enough journals to persistent storage. "
               + "Unsynced transactions: " + (txid - synctxid), new Exception());
           runtime.exit(1);
         }
@@ -966,7 +970,7 @@ public class FSEditLog  {
     
     LOG.info("Registering new backup node: " + bnReg);
     BackupJournalManager bjm = new BackupJournalManager(bnReg, nnReg);
-    journalSet.add(bjm);
+    journalSet.add(bjm, true);
   }
   
   synchronized void releaseBackupStream(NamenodeRegistration registration)

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

@@ -759,7 +759,7 @@ public class FSImage implements Closeable {
    * FSImageSaver assumes that it was launched from a thread that holds
    * FSNamesystem lock and waits for the execution of FSImageSaver thread
    * to finish.
-   * This way we are guraranteed that the namespace is not being updated
+   * This way we are guaranteed that the namespace is not being updated
    * while multiple instances of FSImageSaver are traversing it
    * and writing it out.
    */

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -33,6 +33,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
@@ -547,6 +548,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public static Collection<URI> getNamespaceDirs(Configuration conf) {
     return getStorageDirs(conf, DFS_NAMENODE_NAME_DIR_KEY);
   }
+  
+  public static Collection<URI> getRequiredNamespaceEditsDirs(Configuration conf) {
+    return getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY);
+  }
 
   private static Collection<URI> getStorageDirs(Configuration conf,
                                                 String propertyName) {
@@ -581,7 +586,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public static Collection<URI> getNamespaceEditsDirs(Configuration conf) {
     Collection<URI> editsDirs = getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_KEY);
     editsDirs.addAll(getSharedEditsDirs(conf));
-    return editsDirs;
+    if (editsDirs.isEmpty()) {
+      // If this is the case, no edit dirs have been explicitly configured.
+      // Image dirs are to be used for edits too.
+      return getNamespaceDirs(conf);
+    } else {
+      return editsDirs;
+    }
   }
   
   /**

+ 34 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java

@@ -50,17 +50,16 @@ public class JournalSet implements JournalManager {
    * 
    * If a Journal gets disabled due to an error writing to its
    * stream, then the stream will be aborted and set to null.
-   * 
-   * This should be used outside JournalSet only for testing.
    */
-  @VisibleForTesting
-  static class JournalAndStream {
+  static class JournalAndStream implements CheckableNameNodeResource {
     private final JournalManager journal;
     private boolean disabled = false;
     private EditLogOutputStream stream;
+    private boolean required = false;
     
-    public JournalAndStream(JournalManager manager) {
+    public JournalAndStream(JournalManager manager, boolean required) {
       this.journal = manager;
+      this.required = required;
     }
 
     public void startLogSegment(long txId) throws IOException {
@@ -132,9 +131,24 @@ public class JournalSet implements JournalManager {
     private void setDisabled(boolean disabled) {
       this.disabled = disabled;
     }
+    
+    @Override
+    public boolean isResourceAvailable() {
+      return !isDisabled();
+    }
+    
+    @Override
+    public boolean isRequired() {
+      return required;
+    }
   }
   
   private List<JournalAndStream> journals = Lists.newArrayList();
+  final int minimumRedundantJournals;
+  
+  JournalSet(int minimumRedundantResources) {
+    this.minimumRedundantJournals = minimumRedundantResources;
+  }
   
   @Override
   public EditLogOutputStream startLogSegment(final long txId) throws IOException {
@@ -232,16 +246,15 @@ public class JournalSet implements JournalManager {
   }
 
   /**
-   * Returns true if there are no journals or all are disabled.
-   * @return True if no journals or all are disabled.
+   * Returns true if there are no journals, all redundant journals are disabled,
+   * or any required journals are disabled.
+   * 
+   * @return True if there no journals, all redundant journals are disabled,
+   * or any required journals are disabled.
    */
   public boolean isEmpty() {
-    for (JournalAndStream jas : journals) {
-      if (!jas.isDisabled()) {
-        return false;
-      }
-    }
-    return true;
+    return !NameNodeResourcePolicy.areResourcesAvailable(journals,
+        minimumRedundantJournals);
   }
   
   /**
@@ -292,9 +305,11 @@ public class JournalSet implements JournalManager {
       }
     }
     disableAndReportErrorOnJournals(badJAS);
-    if (badJAS.size() >= journals.size()) {
-      LOG.error("Error: "+status+" failed for all journals");
-      throw new IOException(status+" failed on all the journals");
+    if (!NameNodeResourcePolicy.areResourcesAvailable(journals,
+        minimumRedundantJournals)) {
+      String message = status + " failed for too many journals";
+      LOG.error("Error: " + message);
+      throw new IOException(message);
     }
   }
   
@@ -450,8 +465,9 @@ public class JournalSet implements JournalManager {
     return jList;
   }
 
-  void add(JournalManager j) {
-    journals.add(new JournalAndStream(j));
+  void add(JournalManager j, boolean required) {
+    JournalAndStream jas = new JournalAndStream(j, required);
+    journals.add(jas);
   }
   
   void remove(JournalManager j) {

+ 106 - 46
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java

@@ -27,6 +27,7 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -40,37 +41,80 @@ import com.google.common.base.Predicate;
  * 
  * NameNodeResourceChecker provides a method -
  * <code>hasAvailableDiskSpace</code> - which will return true if and only if
- * the NameNode has disk space available on all volumes which are configured to
- * be checked. Volumes containing file system name/edits dirs are added by
- * default, and arbitrary extra volumes may be configured as well.
+ * the NameNode has disk space available on all required volumes, and any volume
+ * which is configured to be redundant. Volumes containing file system edits dirs
+ * are added by default, and arbitrary extra volumes may be configured as well.
  */
-public class NameNodeResourceChecker {
+@InterfaceAudience.Private
+class NameNodeResourceChecker {
   private static final Log LOG = LogFactory.getLog(NameNodeResourceChecker.class.getName());
 
   // Space (in bytes) reserved per volume.
   private long duReserved;
 
   private final Configuration conf;
-  private Map<String, DF> volumes;
+  private Map<String, CheckedVolume> volumes;
+  private int minimumRedundantVolumes;
+  
+  @VisibleForTesting
+  class CheckedVolume implements CheckableNameNodeResource {
+    private DF df;
+    private boolean required;
+    private String volume;
+    
+    public CheckedVolume(File dirToCheck, boolean required)
+        throws IOException {
+      df = new DF(dirToCheck, conf);
+      this.required = required;
+      volume = df.getFilesystem();
+    }
+    
+    public String getVolume() {
+      return volume;
+    }
+    
+    @Override
+    public boolean isRequired() {
+      return required;
+    }
+
+    @Override
+    public boolean isResourceAvailable() {
+      long availableSpace = df.getAvailable();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Space available on volume '" + volume + "' is "
+            + availableSpace);
+      }
+      if (availableSpace < duReserved) {
+        LOG.warn("Space available on volume '" + volume + "' is "
+            + availableSpace +
+            ", which is below the configured reserved amount " + duReserved);
+        return false;
+      } else {
+        return true;
+      }
+    }
+    
+    @Override
+    public String toString() {
+      return "volume: " + volume + " required: " + required +
+          " resource available: " + isResourceAvailable();
+    }
+  }
 
   /**
-   * Create a NameNodeResourceChecker, which will check the name dirs and edits
-   * dirs set in <code>conf</code>.
-   * 
-   * @param conf
-   * @throws IOException
+   * Create a NameNodeResourceChecker, which will check the edits dirs and any
+   * additional dirs to check set in <code>conf</code>.
    */
   public NameNodeResourceChecker(Configuration conf) throws IOException {
     this.conf = conf;
-    volumes = new HashMap<String, DF>();
+    volumes = new HashMap<String, CheckedVolume>();
 
     duReserved = conf.getLong(DFSConfigKeys.DFS_NAMENODE_DU_RESERVED_KEY,
         DFSConfigKeys.DFS_NAMENODE_DU_RESERVED_DEFAULT);
-  
+    
     Collection<URI> extraCheckedVolumes = Util.stringCollectionAsURIs(conf
         .getTrimmedStringCollection(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_KEY));
-
-    addDirsToCheck(FSNamesystem.getNamespaceDirs(conf));
     
     Collection<URI> localEditDirs = Collections2.filter(
         FSNamesystem.getNamespaceEditsDirs(conf),
@@ -82,70 +126,86 @@ public class NameNodeResourceChecker {
             return false;
           }
         });
-    addDirsToCheck(localEditDirs);
-    addDirsToCheck(extraCheckedVolumes);
+
+    // Add all the local edits dirs, marking some as required if they are
+    // configured as such.
+    for (URI editsDirToCheck : localEditDirs) {
+      addDirToCheck(editsDirToCheck,
+          FSNamesystem.getRequiredNamespaceEditsDirs(conf).contains(
+              editsDirToCheck));
+    }
+
+    // All extra checked volumes are marked "required"
+    for (URI extraDirToCheck : extraCheckedVolumes) {
+      addDirToCheck(extraDirToCheck, true);
+    }
+    
+    minimumRedundantVolumes = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_KEY,
+        DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_DEFAULT);
   }
 
   /**
-   * Add the passed-in directories to the list of volumes to check.
+   * Add the volume of the passed-in directory to the list of volumes to check.
+   * If <code>required</code> is true, and this volume is already present, but
+   * is marked redundant, it will be marked required. If the volume is already
+   * present but marked required then this method is a no-op.
    * 
-   * @param directoriesToCheck
-   *          The directories whose volumes will be checked for available space.
-   * @throws IOException
+   * @param directoryToCheck
+   *          The directory whose volume will be checked for available space.
    */
-  private void addDirsToCheck(Collection<URI> directoriesToCheck)
+  private void addDirToCheck(URI directoryToCheck, boolean required)
       throws IOException {
-    for (URI directoryUri : directoriesToCheck) {
-      File dir = new File(directoryUri.getPath());
-      if (!dir.exists()) {
-        throw new IOException("Missing directory "+dir.getAbsolutePath());
-      }
-      DF df = new DF(dir, conf);
-      volumes.put(df.getFilesystem(), df);
+    File dir = new File(directoryToCheck.getPath());
+    if (!dir.exists()) {
+      throw new IOException("Missing directory "+dir.getAbsolutePath());
+    }
+    
+    CheckedVolume newVolume = new CheckedVolume(dir, required);
+    CheckedVolume volume = volumes.get(newVolume.getVolume());
+    if (volume == null || (volume != null && !volume.isRequired())) {
+      volumes.put(newVolume.getVolume(), newVolume);
     }
   }
 
   /**
    * Return true if disk space is available on at least one of the configured
-   * volumes.
+   * redundant volumes, and all of the configured required volumes.
    * 
    * @return True if the configured amount of disk space is available on at
-   *         least one volume, false otherwise.
-   * @throws IOException
+   *         least one redundant volume and all of the required volumes, false
+   *         otherwise.
    */
   boolean hasAvailableDiskSpace()
       throws IOException {
-    return getVolumesLowOnSpace().size() < volumes.size();
+    return NameNodeResourcePolicy.areResourcesAvailable(volumes.values(),
+        minimumRedundantVolumes);
   }
 
   /**
    * Return the set of directories which are low on space.
+   * 
    * @return the set of directories whose free space is below the threshold.
-   * @throws IOException 
    */
+  @VisibleForTesting
   Collection<String> getVolumesLowOnSpace() throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Going to check the following volumes disk space: " + volumes);
     }
     Collection<String> lowVolumes = new ArrayList<String>();
-    for (DF volume : volumes.values()) {
-      long availableSpace = volume.getAvailable();
-      String fileSystem = volume.getFilesystem();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Space available on volume '" + fileSystem + "' is " + availableSpace);
-      }
-      if (availableSpace < duReserved) {
-        LOG.warn("Space available on volume '" + fileSystem + "' is "
-            + availableSpace +
-            ", which is below the configured reserved amount " + duReserved);
-        lowVolumes.add(volume.getFilesystem());
-      }
+    for (CheckedVolume volume : volumes.values()) {
+      lowVolumes.add(volume.getVolume());
     }
     return lowVolumes;
   }
   
   @VisibleForTesting
-  void setVolumes(Map<String, DF> volumes) {
+  void setVolumes(Map<String, CheckedVolume> volumes) {
     this.volumes = volumes;
   }
+  
+  @VisibleForTesting
+  void setMinimumReduntdantVolumes(int minimumRedundantVolumes) {
+    this.minimumRedundantVolumes = minimumRedundantVolumes;
+  }
 }

+ 81 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java

@@ -0,0 +1,81 @@
+/**
+ * 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.namenode;
+
+import java.util.Collection;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Given a set of checkable resources, this class is capable of determining
+ * whether sufficient resources are available for the NN to continue operating.
+ */
+@InterfaceAudience.Private
+final class NameNodeResourcePolicy {
+
+  /**
+   * Return true if and only if there are sufficient NN
+   * resources to continue logging edits.
+   * 
+   * @param resources the collection of resources to check.
+   * @param minimumRedundantResources the minimum number of redundant resources
+   *        required to continue operation.
+   * @return true if and only if there are sufficient NN resources to
+   *         continue logging edits.
+   * @throws RuntimeException if the number of <bold>configured</bold>
+   *         redundant resources is fewer than the minimum number of available
+   *         redundant resources.
+   */
+  static boolean areResourcesAvailable(
+      Collection<? extends CheckableNameNodeResource> resources,
+      int minimumRedundantResources) {
+    
+    int requiredResourceCount = 0;
+    int redundantResourceCount = 0;
+    int disabledRedundantResourceCount = 0;
+    for (CheckableNameNodeResource resource : resources) {
+      if (!resource.isRequired()) {
+        redundantResourceCount++;
+        if (!resource.isResourceAvailable()) {
+          disabledRedundantResourceCount++;
+        }
+      } else {
+        requiredResourceCount++;
+        if (!resource.isResourceAvailable()) {
+          // Short circuit - a required resource is not available.
+          return false;
+        }
+      }
+    }
+    
+    if (redundantResourceCount < minimumRedundantResources) {
+      throw new RuntimeException("Need a minimum of " + minimumRedundantResources
+          + " for NN to operate but only " + redundantResourceCount
+          + " are configured.");
+    }
+    
+    if (redundantResourceCount == 0) {
+      // If there are no redundant resources, return true if there are any
+      // required resources available.
+      return requiredResourceCount > 0;
+    } else {
+      return redundantResourceCount - disabledRedundantResourceCount >=
+          minimumRedundantResources;
+    }
+  }
+}

+ 122 - 0
hadoop-hdfs-project/hadoop-hdfs/src/proto/ClientDatanodeProtocol.proto

@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "ClientDatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * block - block for which visible length is requested
+ */
+message GetReplicaVisibleLengthRequestProto {
+  required ExtendedBlockProto block = 1;
+}
+
+/**
+ * length - visible length of the block
+ */
+message GetReplicaVisibleLengthResponseProto {
+  required uint64 length = 1;
+}
+
+/**
+ * void request
+ */
+message RefreshNamenodesRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshNamenodesResponseProto {
+}
+
+/**
+ * blockPool - block pool to be deleted
+ * force - if false, delete the block pool only if it is empty.
+ *         if true, delete the block pool even if it has blocks.
+ */
+message DeleteBlockPoolRequestProto {
+  required string blockPool = 1;
+  required bool force = 2;
+}
+
+/**
+ * void response
+ */
+message DeleteBlockPoolResponseProto {
+}
+
+/**
+ * Gets the file information where block and its metadata is stored
+ * block - block for which path information is being requested
+ * token - block token
+ */
+message GetBlockLocalPathInfoRequestProto {
+  required ExtendedBlockProto block = 1;
+  required BlockTokenIdentifierProto token = 2;
+}
+
+/**
+ * block - block for which file path information is being returned
+ * localPath - file path where the block data is stored
+ * localMetaPath - file path where the block meta data is stored
+ */
+message GetBlockLocalPathInfoResponseProto {
+  required ExtendedBlockProto block = 1;
+  required string localPath = 2;
+  required string localMetaPath = 3;
+}
+
+/**
+ * Protocol used from client to the Datanode.
+ * See the request and response for details of rpc call.
+ */
+service ClientDatanodeProtocolService {
+  /**
+   * Returns the visible length of the replica
+   */
+  rpc getReplicaVisibleLength(GetReplicaVisibleLengthRequestProto)
+      returns(GetReplicaVisibleLengthResponseProto);
+
+  /**
+   * Refresh the list of federated namenodes from updated configuration.
+   * Adds new namenodes and stops the deleted namenodes.
+   */
+  rpc refreshNamenode(RefreshNamenodesRequestProto)
+      returns(RefreshNamenodesResponseProto);
+
+  /**
+   * Delete the block pool from the datanode.
+   */
+  rpc deleteBlockPool(DeleteBlockPoolRequestProto)
+      returns(DeleteBlockPoolResponseProto);
+
+  /**
+   * Retrieves the path names of the block file and metadata file stored on the
+   * local file system.
+   */
+  rpc getBlockLocalPathInfo(GetBlockLocalPathInfoRequestProto)
+      returns(GetBlockLocalPathInfoResponseProto);
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/proto/InterDatanodeProtocol.proto

@@ -38,7 +38,7 @@ message InitReplicaRecoveryRequestProto {
  * Repica recovery information
  */
 message InitReplicaRecoveryResponseProto {
-  required ReplicaState state = 1; // State fo the replica
+  required ReplicaStateProto state = 1; // State of the replica
   required BlockProto block = 2;   // block information
 }
 

+ 21 - 3
hadoop-hdfs-project/hadoop-hdfs/src/proto/NamenodeProtocol.proto

@@ -42,7 +42,7 @@ message GetBlocksRequestProto {
  * blocks - List of returned blocks
  */
 message GetBlocksResponseProto {
-  required BlockWithLocationsProto blocks = 1; // List of blocks
+  required BlocksWithLocationsProto blocks = 1; // List of blocks
 }
 
 /**
@@ -85,12 +85,25 @@ message RollEditLogResponseProto {
 }
 
 /**
- * registartion - Namenode reporting the error
+ * void request
+ */
+message VersionRequestProto {
+}
+
+/**
+ * void request
+ */
+message VersionResponseProto {
+  required NamespaceInfoProto info = 1;
+}
+
+/**
+ * registration - 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 NamenodeRegistrationProto registration = 1; // Registration info
   required uint32 errorCode = 2;  // Error code
   required string msg = 3;        // Error message
 }
@@ -193,6 +206,11 @@ service NamenodeProtocolService {
    */
   rpc rollEditLog(RollEditLogRequestProto) returns(RollEditLogResponseProto);
 
+  /**
+   * Close the current editlog and open a new one for checkpointing purposes
+   */
+  rpc versionRequest(VersionRequestProto) returns(VersionResponseProto);
+
   /**
    * Report from a sub-ordinate namenode of an error to the active namenode.
    * Active namenode may decide to unregister the reporting namenode 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/proto/hdfs.proto

@@ -270,8 +270,8 @@ message BlockProto {
  * Block and datanodes where is it located
  */
 message BlockWithLocationsProto {
-  required BlockProto block = 1;            // Block
-  repeated DatanodeIDProto datanodeIDs = 2; // Datanodes with replicas of the block
+  required BlockProto block = 1;   // Block
+  repeated string datanodeIDs = 2; // Datanodes with replicas of the block
 }
 
 /**
@@ -329,7 +329,7 @@ message ExportedBlockKeysProto {
 /**
  * State of a block replica at a datanode
  */
-enum ReplicaState {
+enum ReplicaStateProto {
   FINALIZED = 0;  // State of a replica when it is not modified
   RBW = 1;        // State of replica that is being written to
   RWR = 2;        // State of replica that is waiting to be recovered

+ 253 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java

@@ -19,12 +19,46 @@ package org.apache.hadoop.hdfs.protocolPB;
 
 import static junit.framework.Assert.*;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
+import org.apache.hadoop.hdfs.security.token.block.BlockKey;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
 import org.junit.Test;
 
 /**
@@ -46,10 +80,14 @@ public class TestPBHelper {
     assertEquals(NamenodeRole.NAMENODE,
         PBHelper.convert(NamenodeRoleProto.NAMENODE));
   }
-  
+
+  private static StorageInfo getStorageInfo() {
+    return new StorageInfo(1, 2, "cid", 3);
+  }
+
   @Test
   public void testConvertStoragInfo() {
-    StorageInfo info = new StorageInfo(1, 2, "cid", 3);
+    StorageInfo info = getStorageInfo();
     StorageInfoProto infoProto = PBHelper.convert(info);
     StorageInfo info2 = PBHelper.convert(infoProto);
     assertEquals(info.getClusterID(), info2.getClusterID());
@@ -57,10 +95,10 @@ public class TestPBHelper {
     assertEquals(info.getLayoutVersion(), info2.getLayoutVersion());
     assertEquals(info.getNamespaceID(), info2.getNamespaceID());
   }
-  
+
   @Test
   public void testConvertNamenodeRegistration() {
-    StorageInfo info = new StorageInfo(1, 2, "cid", 3);
+    StorageInfo info = getStorageInfo();
     NamenodeRegistration reg = new NamenodeRegistration("address:999",
         "http:1000", info, NamenodeRole.NAMENODE);
     NamenodeRegistrationProto regProto = PBHelper.convert(reg);
@@ -74,6 +112,217 @@ public class TestPBHelper {
     assertEquals(reg.getRegistrationID(), reg2.getRegistrationID());
     assertEquals(reg.getRole(), reg2.getRole());
     assertEquals(reg.getVersion(), reg2.getVersion());
+
+  }
+
+  @Test
+  public void testConvertDatanodeID() {
+    DatanodeID dn = new DatanodeID("node", "sid", 1, 2);
+    DatanodeIDProto dnProto = PBHelper.convert(dn);
+    DatanodeID dn2 = PBHelper.convert(dnProto);
+    assertEquals(dn.getHost(), dn2.getHost());
+    assertEquals(dn.getInfoPort(), dn2.getInfoPort());
+    assertEquals(dn.getIpcPort(), dn2.getIpcPort());
+    assertEquals(dn.getName(), dn2.getName());
+    assertEquals(dn.getPort(), dn2.getPort());
+    assertEquals(dn.getStorageID(), dn2.getStorageID());
+  }
+
+  @Test
+  public void testConvertBlock() {
+    Block b = new Block(1, 100, 3);
+    BlockProto bProto = PBHelper.convert(b);
+    Block b2 = PBHelper.convert(bProto);
+    assertEquals(b, b2);
+  }
+
+  private static BlockWithLocations getBlockWithLocations(int bid) {
+    return new BlockWithLocations(new Block(bid, 0, 1), new String[] { "dn1",
+        "dn2", "dn3" });
+  }
+
+  private void compare(BlockWithLocations locs1, BlockWithLocations locs2) {
+    assertEquals(locs1.getBlock(), locs2.getBlock());
+    assertTrue(Arrays.equals(locs1.getDatanodes(), locs2.getDatanodes()));
+  }
+
+  @Test
+  public void testConvertBlockWithLocations() {
+    BlockWithLocations locs = getBlockWithLocations(1);
+    BlockWithLocationsProto locsProto = PBHelper.convert(locs);
+    BlockWithLocations locs2 = PBHelper.convert(locsProto);
+    compare(locs, locs2);
+  }
+
+  @Test
+  public void testConvertBlocksWithLocations() {
+    BlockWithLocations[] list = new BlockWithLocations[] {
+        getBlockWithLocations(1), getBlockWithLocations(2) };
+    BlocksWithLocations locs = new BlocksWithLocations(list);
+    BlocksWithLocationsProto locsProto = PBHelper.convert(locs);
+    BlocksWithLocations locs2 = PBHelper.convert(locsProto);
+    BlockWithLocations[] blocks = locs.getBlocks();
+    BlockWithLocations[] blocks2 = locs2.getBlocks();
+    assertEquals(blocks.length, blocks2.length);
+    for (int i = 0; i < blocks.length; i++) {
+      compare(blocks[i], blocks2[i]);
+    }
+  }
+
+  private static BlockKey getBlockKey(int keyId) {
+    return new BlockKey(keyId, 10, "encodedKey".getBytes());
+  }
+
+  private void compare(BlockKey k1, BlockKey k2) {
+    assertEquals(k1.getExpiryDate(), k2.getExpiryDate());
+    assertEquals(k1.getKeyId(), k2.getKeyId());
+    assertTrue(Arrays.equals(k1.getEncodedKey(), k2.getEncodedKey()));
+
+  }
+
+  @Test
+  public void testConvertBlockKey() {
+    BlockKey key = getBlockKey(1);
+    BlockKeyProto keyProto = PBHelper.convert(key);
+    BlockKey key1 = PBHelper.convert(keyProto);
+    compare(key, key1);
+  }
+
+  @Test
+  public void testConvertExportedBlockKeys() {
+    BlockKey[] keys = new BlockKey[] { getBlockKey(2), getBlockKey(3) };
+    ExportedBlockKeys expKeys = new ExportedBlockKeys(true, 9, 10,
+        getBlockKey(1), keys);
+    ExportedBlockKeysProto expKeysProto = PBHelper.convert(expKeys);
+    ExportedBlockKeys expKeys1 = PBHelper.convert(expKeysProto);
+
+    BlockKey[] allKeys = expKeys.getAllKeys();
+    BlockKey[] allKeys1 = expKeys1.getAllKeys();
+    assertEquals(allKeys.length, allKeys1.length);
+    for (int i = 0; i < allKeys.length; i++) {
+      compare(allKeys[i], allKeys1[i]);
+    }
+    compare(expKeys.getCurrentKey(), expKeys1.getCurrentKey());
+    assertEquals(expKeys.getKeyUpdateInterval(),
+        expKeys1.getKeyUpdateInterval());
+    assertEquals(expKeys.getTokenLifetime(), expKeys1.getTokenLifetime());
+  }
+
+  @Test
+  public void testConvertCheckpointSignature() {
+    CheckpointSignature s = new CheckpointSignature(getStorageInfo(), "bpid",
+        100, 1);
+    CheckpointSignatureProto sProto = PBHelper.convert(s);
+    CheckpointSignature s1 = PBHelper.convert(sProto);
+    assertEquals(s.getBlockpoolID(), s1.getBlockpoolID());
+    assertEquals(s.getClusterID(), s1.getClusterID());
+    assertEquals(s.getCTime(), s1.getCTime());
+    assertEquals(s.getCurSegmentTxId(), s1.getCurSegmentTxId());
+    assertEquals(s.getLayoutVersion(), s1.getLayoutVersion());
+    assertEquals(s.getMostRecentCheckpointTxId(),
+        s1.getMostRecentCheckpointTxId());
+    assertEquals(s.getNamespaceID(), s1.getNamespaceID());
+  }
+  
+  private static void compare(RemoteEditLog l1, RemoteEditLog l2) {
+    assertEquals(l1.getEndTxId(), l2.getEndTxId());
+    assertEquals(l1.getStartTxId(), l2.getStartTxId());
+  }
+  
+  @Test
+  public void testConvertRemoteEditLog() {
+    RemoteEditLog l = new RemoteEditLog(1, 100);
+    RemoteEditLogProto lProto = PBHelper.convert(l);
+    RemoteEditLog l1 = PBHelper.convert(lProto);
+    compare(l, l1);
+  }
+  
+  @Test
+  public void testConvertRemoteEditLogManifest() {
+    List<RemoteEditLog> logs = new ArrayList<RemoteEditLog>();
+    logs.add(new RemoteEditLog(1, 10));
+    logs.add(new RemoteEditLog(11, 20));
+    RemoteEditLogManifest m = new RemoteEditLogManifest(logs);
+    RemoteEditLogManifestProto mProto = PBHelper.convert(m);
+    RemoteEditLogManifest m1 = PBHelper.convert(mProto);
     
+    List<RemoteEditLog> logs1 = m1.getLogs();
+    assertEquals(logs.size(), logs1.size());
+    for (int i = 0; i < logs.size(); i++) {
+      compare(logs.get(i), logs1.get(i));
+    }
+  }
+  
+  public ExtendedBlock getExtendedBlock() {
+    return new ExtendedBlock("bpid", 1, 100, 2);
+  }
+  
+  public DatanodeInfo getDNInfo() {
+    return new DatanodeInfo(new DatanodeID("node", "sid", 1, 2));
+  }
+  
+  private void compare(DatanodeInfo dn1, DatanodeInfo dn2) {
+      assertEquals(dn1.getAdminState(), dn2.getAdminState());
+      assertEquals(dn1.getBlockPoolUsed(), dn2.getBlockPoolUsed());
+      assertEquals(dn1.getBlockPoolUsedPercent(), dn2.getBlockPoolUsedPercent());
+      assertEquals(dn1.getCapacity(), dn2.getCapacity());
+      assertEquals(dn1.getDatanodeReport(), dn2.getDatanodeReport());
+      assertEquals(dn1.getDfsUsed(), dn1.getDfsUsed());
+      assertEquals(dn1.getDfsUsedPercent(), dn1.getDfsUsedPercent());
+      assertEquals(dn1.getHost(), dn2.getHost());
+      assertEquals(dn1.getHostName(), dn2.getHostName());
+      assertEquals(dn1.getInfoPort(), dn2.getInfoPort());
+      assertEquals(dn1.getIpcPort(), dn2.getIpcPort());
+      assertEquals(dn1.getLastUpdate(), dn2.getLastUpdate());
+      assertEquals(dn1.getLevel(), dn2.getLevel());
+      assertEquals(dn1.getNetworkLocation(), dn2.getNetworkLocation());
+  }
+  
+  @Test
+  public void testConvertExtendedBlock() {
+    ExtendedBlock b = getExtendedBlock();
+    ExtendedBlockProto bProto = PBHelper.convert(b);
+    ExtendedBlock b1 = PBHelper.convert(bProto);
+    assertEquals(b, b1);
+    
+    b.setBlockId(-1);
+    bProto = PBHelper.convert(b);
+    b1 = PBHelper.convert(bProto);
+    assertEquals(b, b1);
+  }
+  
+  @Test
+  public void testConvertRecoveringBlock() {
+    DatanodeInfo[] dnInfo = new DatanodeInfo[] { getDNInfo(), getDNInfo() };
+    RecoveringBlock b = new RecoveringBlock(getExtendedBlock(), dnInfo, 3);
+    RecoveringBlockProto bProto = PBHelper.convert(b);
+    RecoveringBlock b1 = PBHelper.convert(bProto);
+    assertEquals(b.getBlock(), b1.getBlock());
+    DatanodeInfo[] dnInfo1 = b1.getLocations();
+    assertEquals(dnInfo.length, dnInfo1.length);
+    for (int i=0; i < dnInfo.length; i++) {
+      compare(dnInfo[0], dnInfo1[0]);
+    }
+  }
+  
+  @Test
+  public void testConvertText() {
+    Text t = new Text("abc".getBytes());
+    String s = t.toString();
+    Text t1 = new Text(s);
+    assertEquals(t, t1);
+  }
+  
+  @Test
+  public void testBlockTokenIdentifier() {
+    Token<BlockTokenIdentifier> token = new Token<BlockTokenIdentifier>(
+        "identifier".getBytes(), "password".getBytes(), new Text("kind"),
+        new Text("service"));
+    BlockTokenIdentifierProto tokenProto = PBHelper.convert(token);
+    Token<BlockTokenIdentifier> token2 = PBHelper.convert(tokenProto);
+    assertTrue(Arrays.equals(token.getIdentifier(), token2.getIdentifier()));
+    assertTrue(Arrays.equals(token.getPassword(), token2.getPassword()));
+    assertEquals(token.getKind(), token2.getKind());
+    assertEquals(token.getService(), token2.getService());
   }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java

@@ -47,7 +47,7 @@ public class TestClusterId {
   private String getClusterId(Configuration config) throws IOException {
     // see if cluster id not empty.
     Collection<URI> dirsToFormat = FSNamesystem.getNamespaceDirs(config);
-    Collection<URI> editsToFormat = new ArrayList<URI>(0);
+    Collection<URI> editsToFormat = FSNamesystem.getNamespaceEditsDirs(config);
     FSImage fsImage = new FSImage(config, dirsToFormat, editsToFormat);
     
     Iterator<StorageDirectory> sdit = 

+ 105 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java

@@ -21,28 +21,32 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.atLeast;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+import java.io.File;
 import java.io.IOException;
 
+import org.apache.commons.lang.StringUtils;
 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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.verification.VerificationMode;
 
 public class TestEditLogJournalFailures {
 
   private int editsPerformed = 0;
-  private Configuration conf;
   private MiniDFSCluster cluster;
   private FileSystem fs;
   private Runtime runtime;
@@ -53,8 +57,13 @@ public class TestEditLogJournalFailures {
    */
   @Before
   public void setUpMiniCluster() throws IOException {
-    conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+    setUpMiniCluster(new HdfsConfiguration(), true);
+  }
+  
+  public void setUpMiniCluster(Configuration conf, boolean manageNameDfsDirs)
+      throws IOException {
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+        .manageNameDfsDirs(manageNameDfsDirs).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
     
@@ -64,11 +73,13 @@ public class TestEditLogJournalFailures {
     
     cluster.getNameNode().getFSImage().getEditLog().setRuntimeForTesting(runtime);
   }
-   
+  
   @After
   public void shutDownMiniCluster() throws IOException {
-    fs.close();
-    cluster.shutdown();
+    if (fs != null)
+      fs.close();
+    if (cluster != null)
+      cluster.shutdown();
   }
    
   @Test
@@ -109,7 +120,7 @@ public class TestEditLogJournalFailures {
     assertTrue(doAnEdit());
     // The previous edit could not be synced to any persistent storage, should
     // have halted the NN.
-    assertExitInvocations(1);
+    assertExitInvocations(atLeast(1));
   }
   
   @Test
@@ -124,6 +135,80 @@ public class TestEditLogJournalFailures {
     assertExitInvocations(0);
     assertFalse(cluster.getNameNode().isInSafeMode());
   }
+  
+  @Test
+  public void testSingleRequiredFailedEditsDirOnSetReadyToFlush()
+      throws IOException {
+    // Set one of the edits dirs to be required.
+    String[] editsDirs = cluster.getConfiguration(0).getTrimmedStrings(
+        DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
+    shutDownMiniCluster();
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY, editsDirs[1]);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY, 0);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_KEY, 0);
+    setUpMiniCluster(conf, true);
+    
+    assertTrue(doAnEdit());
+    // Invalidated the one required edits journal.
+    invalidateEditsDirAtIndex(1, false, false);
+    // Make sure runtime.exit(...) hasn't been called at all yet.
+    assertExitInvocations(0);
+    
+    // This will actually return true in the tests, since the NN will not in
+    // fact call Runtime.exit();
+    doAnEdit();
+    
+    // A single failure of a required journal should result in a call to
+    // runtime.exit(...).
+    assertExitInvocations(atLeast(1));
+  }
+  
+  @Test
+  public void testMultipleRedundantFailedEditsDirOnSetReadyToFlush()
+      throws IOException {
+    // Set up 4 name/edits dirs.
+    shutDownMiniCluster();
+    Configuration conf = new HdfsConfiguration();
+    String[] nameDirs = new String[4];
+    for (int i = 0; i < nameDirs.length; i++) {
+      File nameDir = new File(System.getProperty("test.build.data"),
+          "name-dir" + i);
+      nameDir.mkdirs();
+      nameDirs[i] = nameDir.getAbsolutePath();
+    }
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        StringUtils.join(nameDirs, ","));
+    
+    // Keep running unless there are less than 2 edits dirs remaining.
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY, 2);
+    setUpMiniCluster(conf, false);
+    
+    // All journals active.
+    assertTrue(doAnEdit());
+    assertExitInvocations(0);
+    
+    // Invalidate 1/4 of the redundant journals.
+    invalidateEditsDirAtIndex(0, false, false);
+    assertTrue(doAnEdit());
+    assertExitInvocations(0);
+
+    // Invalidate 2/4 of the redundant journals.
+    invalidateEditsDirAtIndex(1, false, false);
+    assertTrue(doAnEdit());
+    assertExitInvocations(0);
+    
+    // Invalidate 3/4 of the redundant journals.
+    invalidateEditsDirAtIndex(2, false, false);
+    
+    // This will actually return true in the tests, since the NN will not in
+    // fact call Runtime.exit();
+    doAnEdit();
+    
+    // A failure of more than the minimum number of redundant journals should
+    // result in a call to runtime.exit(...).
+    assertExitInvocations(atLeast(1));
+  }
 
   /**
    * Replace the journal at index <code>index</code> with one that throws an
@@ -181,6 +266,17 @@ public class TestEditLogJournalFailures {
   private boolean doAnEdit() throws IOException {
     return fs.mkdirs(new Path("/tmp", Integer.toString(editsPerformed++)));
   }
+  
+  /**
+   * Make sure that Runtime.exit(...) has been called exactly
+   * <code>expectedExits<code> number of times.
+   * 
+   * @param expectedExits the exact number of times Runtime.exit(...) should
+   *                      have been called.
+   */
+  private void assertExitInvocations(int expectedExits) {
+    assertExitInvocations(times(expectedExits));
+  }
 
   /**
    * Make sure that Runtime.exit(...) has been called
@@ -188,7 +284,7 @@ public class TestEditLogJournalFailures {
    * 
    * @param expectedExits the number of times Runtime.exit(...) should have been called.
    */
-  private void assertExitInvocations(int expectedExits) {
-    verify(runtime, times(expectedExits)).exit(anyInt());
+  private void assertExitInvocations(VerificationMode expectedExits) {
+    verify(runtime, expectedExits).exit(anyInt());
   }
 }

+ 177 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java

@@ -0,0 +1,177 @@
+/**
+ * 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.namenode;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.junit.Assert.*;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.io.Writable;
+
+import java.net.URI;
+import java.io.IOException;
+
+public class TestGenericJournalConf {
+  /** 
+   * Test that an exception is thrown if a journal class doesn't exist
+   * in the configuration 
+   */
+  @Test(expected=IllegalArgumentException.class)
+  public void testNotConfigured() throws Exception {
+    MiniDFSCluster cluster = null;
+    Configuration conf = new Configuration();
+
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+             "dummy://test");
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      cluster.waitActive();
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test that an exception is thrown if a journal class doesn't
+   * exist in the classloader.
+   */
+  @Test(expected=IllegalArgumentException.class)
+  public void testClassDoesntExist() throws Exception {
+    MiniDFSCluster cluster = null;
+    Configuration conf = new Configuration();
+
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX + ".dummy",
+             "org.apache.hadoop.nonexistent");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+             "dummy://test");
+
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      cluster.waitActive();
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test that a implementation of JournalManager without a 
+   * (Configuration,URI) constructor throws an exception
+   */
+  @Test
+  public void testBadConstructor() throws Exception {
+    MiniDFSCluster cluster = null;
+    Configuration conf = new Configuration();
+    
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX + ".dummy",
+             BadConstructorJournalManager.class.getName());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+             "dummy://test");
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      cluster.waitActive();
+      fail("Should have failed before this point");
+    } catch (IllegalArgumentException iae) {
+      if (!iae.getMessage().contains("Unable to construct journal")) {
+        fail("Should have failed with unable to construct exception");
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test that a dummy implementation of JournalManager can
+   * be initialized on startup
+   */
+  @Test
+  public void testDummyJournalManager() throws Exception {
+    MiniDFSCluster cluster = null;
+    Configuration conf = new Configuration();
+
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX + ".dummy",
+             DummyJournalManager.class.getName());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+             "dummy://test");
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_KEY, 0);
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      cluster.waitActive();
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  public static class DummyJournalManager implements JournalManager {
+    public DummyJournalManager(Configuration conf, URI u) {}
+    
+    @Override
+    public EditLogOutputStream startLogSegment(long txId) throws IOException {
+      return mock(EditLogOutputStream.class);
+    }
+    
+    @Override
+    public void finalizeLogSegment(long firstTxId, long lastTxId)
+        throws IOException {
+      // noop
+    }
+
+    @Override
+    public EditLogInputStream getInputStream(long fromTxnId)
+        throws IOException {
+      return null;
+    }
+
+    @Override
+    public long getNumberOfTransactions(long fromTxnId)
+        throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void setOutputBufferCapacity(int size) {}
+
+    @Override
+    public void purgeLogsOlderThan(long minTxIdToKeep)
+        throws IOException {}
+
+    @Override
+    public void recoverUnfinalizedSegments() throws IOException {}
+
+    @Override
+    public void close() throws IOException {}
+  }
+
+  public static class BadConstructorJournalManager extends DummyJournalManager {
+    public BadConstructorJournalManager() {
+      super(null, null);
+    }
+  }
+}

+ 51 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java

@@ -19,21 +19,20 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URISyntaxException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NameNodeResourceMonitor;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeResourceChecker.CheckedVolume;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import com.google.common.collect.Lists;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -49,7 +48,7 @@ public class TestNameNodeResourceChecker {
     baseDir = new File(System.getProperty("test.build.data"));
     nameDir = new File(baseDir, "resource-check-name-dir");
     nameDir.mkdirs();
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameDir.getAbsolutePath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameDir.getAbsolutePath());
   }
 
   /**
@@ -90,7 +89,7 @@ public class TestNameNodeResourceChecker {
       throws IOException, InterruptedException {
     MiniDFSCluster cluster = null;
     try {
-      conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameDir.getAbsolutePath());
+      conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameDir.getAbsolutePath());
       conf.setLong(DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY, 1);
       
       cluster = new MiniDFSCluster.Builder(conf)
@@ -145,7 +144,7 @@ public class TestNameNodeResourceChecker {
     File nameDir2 = new File(System.getProperty("test.build.data"), "name-dir2");
     nameDir1.mkdirs();
     nameDir2.mkdirs();
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
         nameDir1.getAbsolutePath() + "," + nameDir2.getAbsolutePath());
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_DU_RESERVED_KEY, Long.MAX_VALUE);
 
@@ -164,7 +163,7 @@ public class TestNameNodeResourceChecker {
     Configuration conf = new Configuration();
     File nameDir = new File(System.getProperty("test.build.data"), "name-dir");
     nameDir.mkdirs();
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameDir.getAbsolutePath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameDir.getAbsolutePath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_KEY, nameDir.getAbsolutePath());
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_DU_RESERVED_KEY, Long.MAX_VALUE);
 
@@ -176,38 +175,70 @@ public class TestNameNodeResourceChecker {
 
   /**
    * Test that the NN is considered to be out of resources only once all
-   * configured volumes are low on resources.
+   * redundant configured volumes are low on resources, or when any required
+   * volume is low on resources. 
    */
   @Test
-  public void testLowResourceVolumePolicy() throws IOException {
+  public void testLowResourceVolumePolicy() throws IOException, URISyntaxException {
     Configuration conf = new Configuration();
     File nameDir1 = new File(System.getProperty("test.build.data"), "name-dir1");
     File nameDir2 = new File(System.getProperty("test.build.data"), "name-dir2");
     nameDir1.mkdirs();
     nameDir2.mkdirs();
     
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
         nameDir1.getAbsolutePath() + "," + nameDir2.getAbsolutePath());
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_KEY, 2);
     
     NameNodeResourceChecker nnrc = new NameNodeResourceChecker(conf);
     
     // For the purpose of this test, we need to force the name dirs to appear to
     // be on different volumes.
-    Map<String, DF> volumes = new HashMap<String, DF>();
-    volumes.put("volume1", new DF(nameDir1, conf));
-    volumes.put("volume2", new DF(nameDir2, conf));
+    Map<String, CheckedVolume> volumes = new HashMap<String, CheckedVolume>();
+    CheckedVolume volume1 = Mockito.mock(CheckedVolume.class);
+    CheckedVolume volume2 = Mockito.mock(CheckedVolume.class);
+    CheckedVolume volume3 = Mockito.mock(CheckedVolume.class);
+    CheckedVolume volume4 = Mockito.mock(CheckedVolume.class);
+    CheckedVolume volume5 = Mockito.mock(CheckedVolume.class);
+    Mockito.when(volume1.isResourceAvailable()).thenReturn(true);
+    Mockito.when(volume2.isResourceAvailable()).thenReturn(true);
+    Mockito.when(volume3.isResourceAvailable()).thenReturn(true);
+    Mockito.when(volume4.isResourceAvailable()).thenReturn(true);
+    Mockito.when(volume5.isResourceAvailable()).thenReturn(true);
+    
+    // Make volumes 4 and 5 required.
+    Mockito.when(volume4.isRequired()).thenReturn(true);
+    Mockito.when(volume5.isRequired()).thenReturn(true);
+    
+    volumes.put("volume1", volume1);
+    volumes.put("volume2", volume2);
+    volumes.put("volume3", volume3);
+    volumes.put("volume4", volume4);
+    volumes.put("volume5", volume5);
     nnrc.setVolumes(volumes);
     
-    NameNodeResourceChecker spyNnrc = Mockito.spy(nnrc);
+    // Initially all dirs have space.
+    assertTrue(nnrc.hasAvailableDiskSpace());
+    
+    // 1/3 redundant dir is low on space.
+    Mockito.when(volume1.isResourceAvailable()).thenReturn(false);
+    assertTrue(nnrc.hasAvailableDiskSpace());
     
-    Mockito.when(spyNnrc.getVolumesLowOnSpace()).thenReturn(
-        Lists.newArrayList("volume1"));
+    // 2/3 redundant dirs are low on space.
+    Mockito.when(volume2.isResourceAvailable()).thenReturn(false);
+    assertFalse(nnrc.hasAvailableDiskSpace());
     
-    assertTrue(spyNnrc.hasAvailableDiskSpace());
+    // Lower the minimum number of redundant volumes that must be available.
+    nnrc.setMinimumReduntdantVolumes(1);
+    assertTrue(nnrc.hasAvailableDiskSpace());
     
-    Mockito.when(spyNnrc.getVolumesLowOnSpace()).thenReturn(
-        Lists.newArrayList("volume1", "volume2"));
+    // Just one required dir is low on space.
+    Mockito.when(volume3.isResourceAvailable()).thenReturn(false);
+    assertFalse(nnrc.hasAvailableDiskSpace());
     
-    assertFalse(spyNnrc.hasAvailableDiskSpace());
+    // Just the other required dir is low on space.
+    Mockito.when(volume3.isResourceAvailable()).thenReturn(true);
+    Mockito.when(volume4.isResourceAvailable()).thenReturn(false);
+    assertFalse(nnrc.hasAvailableDiskSpace());
   }
 }

+ 107 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java

@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.junit.Test;
+
+public class TestNameNodeResourcePolicy {
+
+  @Test
+  public void testSingleRedundantResource() {
+    assertTrue(testResourceScenario(1, 0, 0, 0, 1));
+    assertFalse(testResourceScenario(1, 0, 1, 0, 1));
+  }
+  
+  @Test
+  public void testSingleRequiredResource() {
+    assertTrue(testResourceScenario(0, 1, 0, 0, 0));
+    assertFalse(testResourceScenario(0, 1, 0, 1, 0));
+  }
+  
+  @Test
+  public void testMultipleRedundantResources() {
+    assertTrue(testResourceScenario(4, 0, 0, 0, 4));
+    assertFalse(testResourceScenario(4, 0, 1, 0, 4));
+    assertTrue(testResourceScenario(4, 0, 1, 0, 3));
+    assertFalse(testResourceScenario(4, 0, 2, 0, 3));
+    assertTrue(testResourceScenario(4, 0, 2, 0, 2));
+    assertFalse(testResourceScenario(4, 0, 3, 0, 2));
+    assertTrue(testResourceScenario(4, 0, 3, 0, 1));
+    assertFalse(testResourceScenario(4, 0, 4, 0, 1));
+    try {
+      testResourceScenario(1, 0, 0, 0, 2);
+      fail("Should fail if there are more minimum redundant resources than " +
+          "total redundant resources");
+    } catch (RuntimeException rte) {
+      assertTrue(rte.getMessage().startsWith("Need a minimum"));
+    }
+  }
+  
+  @Test
+  public void testMultipleRequiredResources() {
+    assertTrue(testResourceScenario(0, 3, 0, 0, 0));
+    assertFalse(testResourceScenario(0, 3, 0, 1, 0));
+    assertFalse(testResourceScenario(0, 3, 0, 2, 0));
+    assertFalse(testResourceScenario(0, 3, 0, 3, 0));
+  }
+  
+  @Test
+  public void testRedundantWithRequiredResources() {
+    assertTrue(testResourceScenario(2, 2, 0, 0, 1));
+    assertTrue(testResourceScenario(2, 2, 1, 0, 1));
+    assertFalse(testResourceScenario(2, 2, 2, 0, 1));
+    assertFalse(testResourceScenario(2, 2, 0, 1, 1));
+    assertFalse(testResourceScenario(2, 2, 1, 1, 1));
+    assertFalse(testResourceScenario(2, 2, 2, 1, 1));
+  }
+  
+  private static boolean testResourceScenario(
+      int numRedundantResources,
+      int numRequiredResources,
+      int numFailedRedundantResources,
+      int numFailedRequiredResources,
+      int minimumRedundantResources) {
+    
+    Collection<CheckableNameNodeResource> resources =
+        new ArrayList<CheckableNameNodeResource>();
+    
+    for (int i = 0; i < numRedundantResources; i++) {
+      CheckableNameNodeResource r = mock(CheckableNameNodeResource.class);
+      when(r.isRequired()).thenReturn(false);
+      when(r.isResourceAvailable()).thenReturn(i >= numFailedRedundantResources);
+      resources.add(r);
+    }
+    
+    for (int i = 0; i < numRequiredResources; i++) {
+      CheckableNameNodeResource r = mock(CheckableNameNodeResource.class);
+      when(r.isRequired()).thenReturn(true);
+      when(r.isResourceAvailable()).thenReturn(i >= numFailedRequiredResources);
+      resources.add(r);
+    }
+    
+    return NameNodeResourcePolicy.areResourcesAvailable(resources,
+        minimumRedundantResources);    
+  }
+}

+ 12 - 3
hadoop-mapreduce-project/CHANGES.txt

@@ -40,9 +40,9 @@ Trunk (unreleased changes)
     MAPREDUCE-3415. improve MiniMRYarnCluster & DistributedShell JAR resolution.
     (tucu)
 
-		HADOOP-7862   MR changes to work with HADOOP 7862: 
-									Move the support for multiple protocols to lower layer so that Writable,
-								  PB and Avro can all use it (Sanjay)
+    HADOOP-7862   MR changes to work with HADOOP 7862: 
+    Move the support for multiple protocols to lower layer so that Writable,
+    PB and Avro can all use it (Sanjay)
 
   BUG FIXES
     MAPREDUCE-3412. Fix 'ant docs'. (amarrk)
@@ -74,6 +74,9 @@ Trunk (unreleased changes)
 
     MAPREDUCE-3500. MRJobConfig creates an LD_LIBRARY_PATH using the platform ARCH. (tucu)
 
+    MAPREDUCE-3389. MRApps loads the 'mrapp-generated-classpath' file with 
+    classpath from the build machine. (tucu)
+
 Release 0.23.1 - Unreleased
 
   INCOMPATIBLE CHANGES
@@ -243,6 +246,12 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3485. DISKS_FAILED -101 error code should be defined in same location as 
     ABORTED_CONTAINER_EXIT_STATUS. (Ravi Gummadi via mahadev)
 
+    MAPREDUCE-3496. Fixed client to print queue acls in consistent order.
+    (Jonathan Eagles via acmurthy) 
+
+    MAPREDUCE-3147. Handle leaf queues with the same name properly. (Ravi Prakash via
+    mahadev)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 14 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java

@@ -22,6 +22,7 @@ import static org.apache.hadoop.yarn.util.StringHelper._join;
 import static org.apache.hadoop.yarn.util.StringHelper._split;
 
 import java.io.BufferedReader;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
@@ -180,18 +181,23 @@ public class MRApps extends Apps {
       String mrAppGeneratedClasspathFile = "mrapp-generated-classpath";
       classpathFileStream =
           thisClassLoader.getResourceAsStream(mrAppGeneratedClasspathFile);
-      reader = new BufferedReader(new InputStreamReader(classpathFileStream));
-      String cp = reader.readLine();
-      if (cp != null) {
-        Apps.addToEnvironment(environment, Environment.CLASSPATH.name(), cp.trim());
-      }
       // Put the file itself on classpath for tasks.
+      String classpathElement = thisClassLoader.getResource(mrAppGeneratedClasspathFile).getFile();
+      if (classpathElement.contains("!")) {
+        classpathElement = classpathElement.substring(0, classpathElement.indexOf("!"));
+      }
+      else {
+        classpathElement = new File(classpathElement).getParent();
+      }
       Apps.addToEnvironment(
           environment,
-          Environment.CLASSPATH.name(),
-          thisClassLoader.getResource(mrAppGeneratedClasspathFile).getFile()
-            .split("!")[0]);
+          Environment.CLASSPATH.name(), classpathElement);
 
+      reader = new BufferedReader(new InputStreamReader(classpathFileStream));
+      String cp = reader.readLine();
+      if (cp != null) {
+        Apps.addToEnvironment(environment, Environment.CLASSPATH.name(), cp.trim());
+      }      
       // Add standard Hadoop classes
       for (String c : ApplicationConstants.APPLICATION_CLASSPATH) {
         Apps.addToEnvironment(environment, Environment.CLASSPATH.name(), c);

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java

@@ -22,6 +22,7 @@ import java.io.PrintWriter;
 import java.io.Writer;
 import java.util.List;
 import java.util.ArrayList;
+import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -200,6 +201,7 @@ class JobQueueClient extends Configured implements Tool {
       for (QueueAclsInfo queueInfo : queueAclsInfoList) {
         System.out.print(queueInfo.getQueueName() + "  ");
         String[] ops = queueInfo.getOperations();
+        Arrays.sort(ops);
         int max = ops.length - 1;
         for (int j = 0; j < ops.length; j++) {
           System.out.print(ops[j].replaceFirst("acl-", ""));

+ 8 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -214,7 +214,8 @@ implements ResourceScheduler, CapacitySchedulerContext {
   private static final QueueHook noop = new QueueHook();
   
   @Lock(CapacityScheduler.class)
-  private void initializeQueues(CapacitySchedulerConfiguration conf) {
+  private void initializeQueues(CapacitySchedulerConfiguration conf)
+    throws IOException {
     root = 
         parseQueue(this, conf, null, ROOT, queues, queues, 
             queueComparator, applicationComparator, noop);
@@ -283,7 +284,7 @@ implements ResourceScheduler, CapacitySchedulerContext {
       Map<String, CSQueue> oldQueues, 
       Comparator<CSQueue> queueComparator,
       Comparator<SchedulerApp> applicationComparator,
-      QueueHook hook) {
+      QueueHook hook) throws IOException {
     CSQueue queue;
     String[] childQueueNames = 
       conf.getQueues((parent == null) ? 
@@ -316,6 +317,11 @@ implements ResourceScheduler, CapacitySchedulerContext {
       parentQueue.setChildQueues(childQueues);
     }
 
+    if(queue instanceof LeafQueue == true && queues.containsKey(queueName)
+      && queues.get(queueName) instanceof LeafQueue == true) {
+      throw new IOException("Two leaf queues were named " + queueName
+        + ". Leaf queue names must be distinct");
+    }
     queues.put(queueName, queue);
 
     LOG.info("Initialized queue: " + queue);

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -44,7 +45,7 @@ public class TestApplicationLimits {
   LeafQueue queue;
   
   @Before
-  public void setUp() {
+  public void setUp() throws IOException {
     CapacitySchedulerConfiguration csConf = 
         new CapacitySchedulerConfiguration();
     setupQueueConfiguration(csConf);

+ 17 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -240,4 +240,21 @@ public class TestCapacityScheduler {
     node.checkResourceUsage();
   }
 
+  /** Test that parseQueue throws an exception when two leaf queues have the
+   *  same name
+ * @throws IOException
+   */
+  @Test(expected=IOException.class)
+  public void testParseQueue() throws IOException {
+    CapacityScheduler cs = new CapacityScheduler();
+
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    setupQueueConfiguration(conf);
+    conf.setQueues(CapacityScheduler.ROOT + ".a.a1", new String[] {"b1"} );
+    conf.setCapacity(CapacityScheduler.ROOT + ".a.a1.b1", 100);
+    conf.setUserLimitFactor(CapacityScheduler.ROOT + ".a.a1.b1", 100.0f);
+
+    cs.reinitialize(conf, null, null);
+  }
+
 }

이 변경점에서 너무 많은 파일들이 변경되어 몇몇 파일들은 표시되지 않았습니다.