瀏覽代碼

Merge r1550130 through r1555020 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1555021 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 年之前
父節點
當前提交
498f9674ff
共有 100 個文件被更改,包括 5462 次插入2023 次删除
  1. 44 2
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 4 0
      hadoop-common-project/hadoop-common/pom.xml
  3. 313 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
  4. 384 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
  5. 76 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java
  6. 145 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/UserProvider.java
  7. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
  8. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
  9. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
  10. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
  11. 14 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcConstants.java
  12. 127 44
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  13. 11 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java
  14. 18 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocket.java
  15. 9 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
  16. 60 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java
  17. 51 36
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  18. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
  19. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
  20. 44 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/net/unix/DomainSocket.c
  21. 41 6
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_user_info.c
  22. 17 0
      hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory
  23. 112 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProvider.java
  24. 191 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
  25. 16 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
  26. 13 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  27. 1 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  28. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
  29. 87 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSourceAdapter.java
  30. 32 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java
  31. 19 4
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
  32. 246 20
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  33. 1 8
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  34. 17 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/CacheFlag.java
  35. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
  36. 18 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
  37. 530 393
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  38. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
  39. 15 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  40. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  41. 89 51
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  42. 39 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  43. 30 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  44. 25 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  45. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
  46. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  47. 35 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java
  48. 12 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  49. 19 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
  50. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java
  51. 25 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
  52. 81 84
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
  53. 17 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
  54. 12 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  55. 37 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  56. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  57. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  58. 39 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  59. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnregisteredNodeException.java
  60. 19 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  61. 24 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  62. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  63. 3 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
  64. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
  65. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
  66. 172 43
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  67. 25 48
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  68. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  69. 91 41
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  70. 33 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
  71. 193 165
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  72. 12 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  73. 186 139
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  74. 59 56
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
  75. 15 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  76. 210 72
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  77. 147 185
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  78. 60 60
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  79. 288 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
  80. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
  81. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
  82. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java
  83. 9 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java
  84. 15 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  85. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  86. 36 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  87. 212 79
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  88. 26 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  89. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  90. 13 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  91. 24 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/CachingStrategy.java
  92. 172 123
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  93. 69 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  94. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  95. 2 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  96. 10 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  97. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/Replica.java
  98. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  99. 9 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
  100. 101 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java

+ 44 - 2
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -105,6 +105,9 @@ Trunk (Unreleased)
 
     HADOOP-9833 move slf4j to version 1.7.5 (Kousuke Saruta via stevel)
 
+    HADOOP-10141. Create KeyProvider API to separate encryption key storage
+    from the applications. (omalley)
+
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
@@ -280,6 +283,8 @@ Trunk (Unreleased)
     HDFS-5471. CacheAdmin -listPools fails when user lacks permissions to view
     all pools (Andrew Wang via Colin Patrick McCabe)
 
+    HADOOP-10044 Improve the javadoc of rpc code (sanjay Radia)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -395,12 +400,27 @@ Release 2.4.0 - UNRELEASED
 
     HADOOP-10102. Update commons IO from 2.1 to 2.4 (Akira Ajisaka via stevel)
 
+    HADOOP-10168. fix javadoc of ReflectionUtils#copy. (Thejas Nair via suresh)
+
+    HADOOP-10164. Allow UGI to login with a known Subject (bobby)
+
+    HADOOP-10169. Remove the unnecessary synchronized in JvmMetrics class.
+    (Liang Xie via jing9) 
+
+    HADOOP-10198. DomainSocket: add support for socketpair.
+    (Colin Patrick McCabe via wang)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
 
-   HADOOP-10047. Add a direct-buffer based apis for compression. (Gopal V
-   via acmurthy)
+    HADOOP-10047. Add a direct-buffer based apis for compression. (Gopal V
+    via acmurthy)
+
+    HADOOP-10172. Cache SASL server factories (daryn)
+
+    HADOOP-10173. Remove UGI from DIGEST-MD5 SASL server creation (daryn via
+    kihwal)
 
   BUG FIXES
 
@@ -465,6 +485,19 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10058. TestMetricsSystemImpl#testInitFirstVerifyStopInvokedImmediately
     fails on trunk (Chen He via jeagles)
 
+    HADOOP-8753. LocalDirAllocator throws "ArithmeticException: / by zero" when
+    there is no available space on configured local dir. (Benoy Antony via hitesh)
+
+    HADOOP-10106. Incorrect thread name in RPC log messages. (Ming Ma via jing9)
+
+    HADOOP-9611 mvn-rpmbuild against google-guice > 3.0 yields missing cglib
+    dependency (Timothy St. Clair via stevel)
+
+    HADOOP-10171. TestRPC fails intermittently on jkd7 (Mit Desai via jeagles)
+
+    HADOOP-10147  HDFS-5678 Upgrade to commons-logging 1.1.3 to avoid potential
+    deadlock in MiniDFSCluster (stevel)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -538,6 +571,15 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10081. Client.setupIOStreams can leak socket resources on exception
     or error (Tsuyoshi OZAWA via jlowe)
 
+    HADOOP-10087. UserGroupInformation.getGroupNames() fails to return primary
+    group first when JniBasedUnixGroupsMappingWithFallback is used (cmccabe)
+
+    HADOOP-10175. Har files system authority should preserve userinfo.
+    (Chuan Liu via cnauroth)
+
+    HADOOP-10090. Jobtracker metrics not updated properly after execution
+    of a mapreduce job. (ivanmi)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -209,6 +209,10 @@
       <artifactId>protobuf-java</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-auth</artifactId>

+ 313 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java

@@ -0,0 +1,313 @@
+/**
+ * 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.crypto.key;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.net.URI;
+import java.security.Key;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * KeyProvider based on Java's KeyStore file format. The file may be stored in
+ * any Hadoop FileSystem using the following name mangling:
+ *  jks://hdfs@nn1.example.com/my/keys.jks -> hdfs://nn1.example.com/my/keys.jks
+ *  jks://file/home/owen/keys.jks -> file:///home/owen/keys.jks
+ *
+ * The password for the keystore is taken from the HADOOP_KEYSTORE_PASSWORD
+ * environment variable with a default of 'none'.
+ *
+ * It is expected for encrypted InputFormats and OutputFormats to copy the keys
+ * from the original provider into the job's Credentials object, which is
+ * accessed via the UserProvider. Therefore, this provider won't be used by
+ * MapReduce tasks.
+ */
+@InterfaceAudience.Private
+public class JavaKeyStoreProvider extends KeyProvider {
+  public static final String SCHEME_NAME = "jceks";
+  public static final String KEYSTORE_PASSWORD_NAME =
+      "HADOOP_KEYSTORE_PASSWORD";
+  public static final String KEYSTORE_PASSWORD_DEFAULT = "none";
+
+  private final URI uri;
+  private final Path path;
+  private final FileSystem fs;
+  private final KeyStore keyStore;
+  private final char[] password;
+  private boolean changed = false;
+
+  private final Map<String, Metadata> cache = new HashMap<String, Metadata>();
+
+  private JavaKeyStoreProvider(URI uri, Configuration conf) throws IOException {
+    this.uri = uri;
+    path = unnestUri(uri);
+    fs = FileSystem.get(conf);
+    // Get the password from the user's environment
+    String pw = System.getenv(KEYSTORE_PASSWORD_NAME);
+    if (pw == null) {
+      pw = KEYSTORE_PASSWORD_DEFAULT;
+    }
+    password = pw.toCharArray();
+    try {
+      keyStore = KeyStore.getInstance(SCHEME_NAME);
+      if (fs.exists(path)) {
+        keyStore.load(fs.open(path), password);
+      } else {
+        // required to create an empty keystore. *sigh*
+        keyStore.load(null, password);
+      }
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't create keystore", e);
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException("Can't load keystore " + path, e);
+    } catch (CertificateException e) {
+      throw new IOException("Can't load keystore " + path, e);
+    }
+  }
+
+  @Override
+  public KeyVersion getKeyVersion(String versionName) throws IOException {
+    SecretKeySpec key = null;
+    try {
+      if (!keyStore.containsAlias(versionName)) {
+        return null;
+      }
+      key = (SecretKeySpec) keyStore.getKey(versionName, password);
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't get key " + versionName + " from " +
+                            path, e);
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException("Can't get algorithm for key " + key + " from " +
+                            path, e);
+    } catch (UnrecoverableKeyException e) {
+      throw new IOException("Can't recover key " + key + " from " + path, e);
+    }
+    return new KeyVersion(versionName, key.getEncoded());
+  }
+
+  @Override
+  public Metadata getMetadata(String name) throws IOException {
+    if (cache.containsKey(name)) {
+      return cache.get(name);
+    }
+    try {
+      if (!keyStore.containsAlias(name)) {
+        return null;
+      }
+      Metadata meta = ((KeyMetadata) keyStore.getKey(name, password)).metadata;
+      cache.put(name, meta);
+      return meta;
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't get metadata for " + name +
+          " from keystore " + path, e);
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException("Can't get algorithm for " + name +
+          " from keystore " + path, e);
+    } catch (UnrecoverableKeyException e) {
+      throw new IOException("Can't recover key for " + name +
+          " from keystore " + path, e);
+    }
+  }
+
+  @Override
+  public KeyVersion createKey(String name, byte[] material,
+                               Options options) throws IOException {
+    try {
+      if (keyStore.containsAlias(name) || cache.containsKey(name)) {
+        throw new IOException("Key " + name + " already exists in " + this);
+      }
+    } catch (KeyStoreException e) {
+      throw new IOException("Problem looking up key " + name + " in " + this,
+          e);
+    }
+    Metadata meta = new Metadata(options.getCipher(), options.getBitLength(),
+        new Date(), 1);
+    if (options.getBitLength() != 8 * material.length) {
+      throw new IOException("Wrong key length. Required " +
+          options.getBitLength() + ", but got " + (8 * material.length));
+    }
+    cache.put(name, meta);
+    String versionName = buildVersionName(name, 0);
+    return innerSetKeyVersion(versionName, material, meta.getCipher());
+  }
+
+  @Override
+  public void deleteKey(String name) throws IOException {
+    Metadata meta = getMetadata(name);
+    if (meta == null) {
+      throw new IOException("Key " + name + " does not exist in " + this);
+    }
+    for(int v=0; v < meta.getVersions(); ++v) {
+      String versionName = buildVersionName(name, v);
+      try {
+        if (keyStore.containsAlias(versionName)) {
+          keyStore.deleteEntry(versionName);
+        }
+      } catch (KeyStoreException e) {
+        throw new IOException("Problem removing " + versionName + " from " +
+            this, e);
+      }
+    }
+    try {
+      if (keyStore.containsAlias(name)) {
+        keyStore.deleteEntry(name);
+      }
+    } catch (KeyStoreException e) {
+      throw new IOException("Problem removing " + name + " from " + this, e);
+    }
+    cache.remove(name);
+    changed = true;
+  }
+
+  KeyVersion innerSetKeyVersion(String versionName, byte[] material,
+                                String cipher) throws IOException {
+    try {
+      keyStore.setKeyEntry(versionName, new SecretKeySpec(material, cipher),
+          password, null);
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't store key " + versionName + " in " + this,
+          e);
+    }
+    changed = true;
+    return new KeyVersion(versionName, material);
+  }
+
+  @Override
+  public KeyVersion rollNewVersion(String name,
+                                    byte[] material) throws IOException {
+    Metadata meta = getMetadata(name);
+    if (meta == null) {
+      throw new IOException("Key " + name + " not found");
+    }
+    if (meta.getBitLength() != 8 * material.length) {
+      throw new IOException("Wrong key length. Required " +
+          meta.getBitLength() + ", but got " + (8 * material.length));
+    }
+    int nextVersion = meta.addVersion();
+    String versionName = buildVersionName(name, nextVersion);
+    return innerSetKeyVersion(versionName, material, meta.getCipher());
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (!changed) {
+      return;
+    }
+    // put all of the updates into the keystore
+    for(Map.Entry<String, Metadata> entry: cache.entrySet()) {
+      try {
+        keyStore.setKeyEntry(entry.getKey(), new KeyMetadata(entry.getValue()),
+            password, null);
+      } catch (KeyStoreException e) {
+        throw new IOException("Can't set metadata key " + entry.getKey(),e );
+      }
+    }
+    // write out the keystore
+    FSDataOutputStream out = fs.create(path, true);
+    try {
+      keyStore.store(out, password);
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't store keystore " + this, e);
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException("No such algorithm storing keystore " + this, e);
+    } catch (CertificateException e) {
+      throw new IOException("Certificate exception storing keystore " + this,
+          e);
+    }
+    out.close();
+    changed = false;
+  }
+
+  @Override
+  public String toString() {
+    return uri.toString();
+  }
+
+  /**
+   * The factory to create JksProviders, which is used by the ServiceLoader.
+   */
+  public static class Factory extends KeyProviderFactory {
+    @Override
+    public KeyProvider createProvider(URI providerName,
+                                      Configuration conf) throws IOException {
+      if (SCHEME_NAME.equals(providerName.getScheme())) {
+        return new JavaKeyStoreProvider(providerName, conf);
+      }
+      return null;
+    }
+  }
+
+  /**
+   * An adapter between a KeyStore Key and our Metadata. This is used to store
+   * the metadata in a KeyStore even though isn't really a key.
+   */
+  public static class KeyMetadata implements Key, Serializable {
+    private Metadata metadata;
+    private final static long serialVersionUID = 8405872419967874451L;
+
+    private KeyMetadata(Metadata meta) {
+      this.metadata = meta;
+    }
+
+    @Override
+    public String getAlgorithm() {
+      return metadata.getCipher();
+    }
+
+    @Override
+    public String getFormat() {
+      return "KeyMetadata";
+    }
+
+    @Override
+    public byte[] getEncoded() {
+      return new byte[0];
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException {
+      byte[] serialized = metadata.serialize();
+      out.writeInt(serialized.length);
+      out.write(serialized);
+    }
+
+    private void readObject(ObjectInputStream in
+                            ) throws IOException, ClassNotFoundException {
+      byte[] buf = new byte[in.readInt()];
+      in.readFully(buf);
+      metadata = new Metadata(buf);
+    }
+
+  }
+}

+ 384 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java

@@ -0,0 +1,384 @@
+/**
+ * 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.crypto.key;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.net.URI;
+import java.util.Date;
+import java.util.List;
+
+import com.google.gson.stream.JsonReader;
+import com.google.gson.stream.JsonWriter;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A provider of secret key material for Hadoop applications. Provides an
+ * abstraction to separate key storage from users of encryption. It
+ * is intended to support getting or storing keys in a variety of ways,
+ * including third party bindings.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class KeyProvider {
+  public static final String DEFAULT_CIPHER_NAME =
+      "hadoop.security.key.default.cipher";
+  public static final String DEFAULT_CIPHER = "AES/CTR/NoPadding";
+  public static final String DEFAULT_BITLENGTH_NAME =
+      "hadoop.security.key.default.bitlength";
+  public static final int DEFAULT_BITLENGTH = 256;
+
+  /**
+   * The combination of both the key version name and the key material.
+   */
+  public static class KeyVersion {
+    private final String versionName;
+    private final byte[] material;
+
+    protected KeyVersion(String versionName,
+                         byte[] material) {
+      this.versionName = versionName;
+      this.material = material;
+    }
+
+    public String getVersionName() {
+      return versionName;
+    }
+
+    public byte[] getMaterial() {
+      return material;
+    }
+
+    public String toString() {
+      StringBuilder buf = new StringBuilder();
+      buf.append("key(");
+      buf.append(versionName);
+      buf.append(")=");
+      if (material == null) {
+        buf.append("null");
+      } else {
+        for(byte b: material) {
+          buf.append(' ');
+          int right = b & 0xff;
+          if (right < 0x10) {
+            buf.append('0');
+          }
+          buf.append(Integer.toHexString(right));
+        }
+      }
+      return buf.toString();
+    }
+  }
+
+  /**
+   * Key metadata that is associated with the key.
+   */
+  public static class Metadata {
+    private final static String CIPHER_FIELD = "cipher";
+    private final static String BIT_LENGTH_FIELD = "bitLength";
+    private final static String CREATED_FIELD = "created";
+    private final static String VERSIONS_FIELD = "versions";
+
+    private final String cipher;
+    private final int bitLength;
+    private final Date created;
+    private int versions;
+
+    protected Metadata(String cipher, int bitLength,
+                       Date created, int versions) {
+      this.cipher = cipher;
+      this.bitLength = bitLength;
+      this.created = created;
+      this.versions = versions;
+    }
+
+    public Date getCreated() {
+      return created;
+    }
+
+    public String getCipher() {
+      return cipher;
+    }
+
+    /**
+     * Get the algorithm from the cipher.
+     * @return the algorithm name
+     */
+    public String getAlgorithm() {
+      int slash = cipher.indexOf('/');
+      if (slash == - 1) {
+        return cipher;
+      } else {
+        return cipher.substring(0, slash);
+      }
+    }
+
+    public int getBitLength() {
+      return bitLength;
+    }
+
+    public int getVersions() {
+      return versions;
+    }
+
+    protected int addVersion() {
+      return versions++;
+    }
+
+    /**
+     * Serialize the metadata to a set of bytes.
+     * @return the serialized bytes
+     * @throws IOException
+     */
+    protected byte[] serialize() throws IOException {
+      ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+      JsonWriter writer = new JsonWriter(new OutputStreamWriter(buffer));
+      writer.beginObject();
+      if (cipher != null) {
+        writer.name(CIPHER_FIELD).value(cipher);
+      }
+      if (bitLength != 0) {
+        writer.name(BIT_LENGTH_FIELD).value(bitLength);
+      }
+      if (created != null) {
+        writer.name(CREATED_FIELD).value(created.getTime());
+      }
+      writer.name(VERSIONS_FIELD).value(versions);
+      writer.endObject();
+      writer.flush();
+      return buffer.toByteArray();
+    }
+
+    /**
+     * Deserialize a new metadata object from a set of bytes.
+     * @param bytes the serialized metadata
+     * @throws IOException
+     */
+    protected Metadata(byte[] bytes) throws IOException {
+      String cipher = null;
+      int bitLength = 0;
+      Date created = null;
+      int versions = 0;
+      JsonReader reader = new JsonReader(new InputStreamReader
+          (new ByteArrayInputStream(bytes)));
+      reader.beginObject();
+      while (reader.hasNext()) {
+        String field = reader.nextName();
+        if (CIPHER_FIELD.equals(field)) {
+          cipher = reader.nextString();
+        } else if (BIT_LENGTH_FIELD.equals(field)) {
+          bitLength = reader.nextInt();
+        } else if (CREATED_FIELD.equals(field)) {
+          created = new Date(reader.nextLong());
+        } else if (VERSIONS_FIELD.equals(field)) {
+          versions = reader.nextInt();
+        }
+      }
+      reader.endObject();
+      this.cipher = cipher;
+      this.bitLength = bitLength;
+      this.created = created;
+      this.versions = versions;
+    }
+  }
+
+  /**
+   * Options when creating key objects.
+   */
+  public static class Options {
+    private String cipher;
+    private int bitLength;
+
+    public Options(Configuration conf) {
+      cipher = conf.get(DEFAULT_CIPHER_NAME, DEFAULT_CIPHER);
+      bitLength = conf.getInt(DEFAULT_BITLENGTH_NAME, DEFAULT_BITLENGTH);
+    }
+
+    public Options setCipher(String cipher) {
+      this.cipher = cipher;
+      return this;
+    }
+
+    public Options setBitLength(int bitLength) {
+      this.bitLength = bitLength;
+      return this;
+    }
+
+    protected String getCipher() {
+      return cipher;
+    }
+
+    protected int getBitLength() {
+      return bitLength;
+    }
+  }
+
+  /**
+   * A helper function to create an options object.
+   * @param conf the configuration to use
+   * @return a new options object
+   */
+  public static Options options(Configuration conf) {
+    return new Options(conf);
+  }
+
+  /**
+   * Get the key material for a specific version of the key. This method is used
+   * when decrypting data.
+   * @param versionName the name of a specific version of the key
+   * @return the key material
+   * @throws IOException
+   */
+  public abstract KeyVersion getKeyVersion(String versionName
+                                            ) throws IOException;
+
+  /**
+   * Get the current version of the key, which should be used for encrypting new
+   * data.
+   * @param name the base name of the key
+   * @return the version name of the current version of the key or null if the
+   *    key version doesn't exist
+   * @throws IOException
+   */
+  public KeyVersion getCurrentKey(String name) throws IOException {
+    Metadata meta = getMetadata(name);
+    if (meta == null) {
+      return null;
+    }
+    return getKeyVersion(buildVersionName(name, meta.getVersions() - 1));
+  }
+
+  /**
+   * Get metadata about the key.
+   * @param name the basename of the key
+   * @return the key's metadata or null if the key doesn't exist
+   * @throws IOException
+   */
+  public abstract Metadata getMetadata(String name) throws IOException;
+
+  /**
+   * Create a new key. The given key must not already exist.
+   * @param name the base name of the key
+   * @param material the key material for the first version of the key.
+   * @param options the options for the new key.
+   * @return the version name of the first version of the key.
+   * @throws IOException
+   */
+  public abstract KeyVersion createKey(String name, byte[] material,
+                                       Options options) throws IOException;
+
+  /**
+   * Delete the given key.
+   * @param name the name of the key to delete
+   * @throws IOException
+   */
+  public abstract void deleteKey(String name) throws IOException;
+
+  /**
+   * Roll a new version of the given key.
+   * @param name the basename of the key
+   * @param material the new key material
+   * @return the name of the new version of the key
+   * @throws IOException
+   */
+  public abstract KeyVersion rollNewVersion(String name,
+                                             byte[] material
+                                            ) throws IOException;
+
+  /**
+   * Ensures that any changes to the keys are written to persistent store.
+   * @throws IOException
+   */
+  public abstract void flush() throws IOException;
+
+  /**
+   * Split the versionName in to a base name. Converts "/aaa/bbb/3" to
+   * "/aaa/bbb".
+   * @param versionName the version name to split
+   * @return the base name of the key
+   * @throws IOException
+   */
+  public static String getBaseName(String versionName) throws IOException {
+    int div = versionName.lastIndexOf('@');
+    if (div == -1) {
+      throw new IOException("No version in key path " + versionName);
+    }
+    return versionName.substring(0, div);
+  }
+
+  /**
+   * Build a version string from a basename and version number. Converts
+   * "/aaa/bbb" and 3 to "/aaa/bbb@3".
+   * @param name the basename of the key
+   * @param version the version of the key
+   * @return the versionName of the key.
+   */
+  protected static String buildVersionName(String name, int version) {
+    return name + "@" + version;
+  }
+
+  /**
+   * Convert a nested URI to decode the underlying path. The translation takes
+   * the authority and parses it into the underlying scheme and authority.
+   * For example, "myscheme://hdfs@nn/my/path" is converted to
+   * "hdfs://nn/my/path".
+   * @param nestedUri the URI from the nested URI
+   * @return the unnested path
+   */
+  public static Path unnestUri(URI nestedUri) {
+    String[] parts = nestedUri.getAuthority().split("@", 2);
+    StringBuilder result = new StringBuilder(parts[0]);
+    result.append("://");
+    if (parts.length == 2) {
+      result.append(parts[1]);
+    }
+    result.append(nestedUri.getPath());
+    if (nestedUri.getQuery() != null) {
+      result.append("?");
+      result.append(nestedUri.getQuery());
+    }
+    if (nestedUri.getFragment() != null) {
+      result.append("#");
+      result.append(nestedUri.getFragment());
+    }
+    return new Path(result.toString());
+  }
+
+  /**
+   * Find the provider with the given key.
+   * @param providerList the list of providers
+   * @param keyName the key name we are looking for
+   * @return the KeyProvider that has the key
+   */
+  public static KeyProvider findProvider(List<KeyProvider> providerList,
+                                         String keyName) throws IOException {
+    for(KeyProvider provider: providerList) {
+      if (provider.getMetadata(keyName) != null) {
+        return provider;
+      }
+    }
+    throw new IOException("Can't find KeyProvider for key " + keyName);
+  }
+}

+ 76 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java

@@ -0,0 +1,76 @@
+/**
+ * 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.crypto.key;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ServiceLoader;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A factory to create a list of KeyProvider based on the path given in a
+ * Configuration. It uses a service loader interface to find the available
+ * KeyProviders and create them based on the list of URIs.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class KeyProviderFactory {
+  public static final String KEY_PROVIDER_PATH =
+      "hadoop.security.key.provider.path";
+
+  public abstract KeyProvider createProvider(URI providerName,
+                                             Configuration conf
+                                             ) throws IOException;
+
+  private static final ServiceLoader<KeyProviderFactory> serviceLoader =
+      ServiceLoader.load(KeyProviderFactory.class);
+
+  public static List<KeyProvider> getProviders(Configuration conf
+                                               ) throws IOException {
+    List<KeyProvider> result = new ArrayList<KeyProvider>();
+    for(String path: conf.getStringCollection(KEY_PROVIDER_PATH)) {
+      try {
+        URI uri = new URI(path);
+        boolean found = false;
+        for(KeyProviderFactory factory: serviceLoader) {
+          KeyProvider kp = factory.createProvider(uri, conf);
+          if (kp != null) {
+            result.add(kp);
+            found = true;
+            break;
+          }
+        }
+        if (!found) {
+          throw new IOException("No KeyProviderFactory for " + uri + " in " +
+              KEY_PROVIDER_PATH);
+        }
+      } catch (URISyntaxException error) {
+        throw new IOException("Bad configuration of " + KEY_PROVIDER_PATH +
+            " at " + path, error);
+      }
+    }
+    return result;
+  }
+}

+ 145 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/UserProvider.java

@@ -0,0 +1,145 @@
+/**
+ * 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.crypto.key;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * A KeyProvider factory for UGIs. It uses the credentials object associated
+ * with the current user to find keys. This provider is created using a
+ * URI of "user:///".
+ */
+@InterfaceAudience.Private
+public class UserProvider extends KeyProvider {
+  public static final String SCHEME_NAME = "user";
+  private final UserGroupInformation user;
+  private final Credentials credentials;
+  private final Map<String, Metadata> cache = new HashMap<String, Metadata>();
+
+  private UserProvider() throws IOException {
+    user = UserGroupInformation.getCurrentUser();
+    credentials = user.getCredentials();
+  }
+
+  @Override
+  public KeyVersion getKeyVersion(String versionName) {
+    byte[] bytes = credentials.getSecretKey(new Text(versionName));
+    if (bytes == null) {
+      return null;
+    }
+    return new KeyVersion(versionName, bytes);
+  }
+
+  @Override
+  public Metadata getMetadata(String name) throws IOException {
+    if (cache.containsKey(name)) {
+      return cache.get(name);
+    }
+    byte[] serialized = credentials.getSecretKey(new Text(name));
+    if (serialized == null) {
+      return null;
+    }
+    Metadata result = new Metadata(serialized);
+    cache.put(name, result);
+    return result;
+  }
+
+  @Override
+  public KeyVersion createKey(String name, byte[] material,
+                               Options options) throws IOException {
+    Text nameT = new Text(name);
+    if (credentials.getSecretKey(nameT) != null) {
+      throw new IOException("Key " + name + " already exists in " + this);
+    }
+    if (options.getBitLength() != 8 * material.length) {
+      throw new IOException("Wrong key length. Required " +
+          options.getBitLength() + ", but got " + (8 * material.length));
+    }
+    Metadata meta = new Metadata(options.getCipher(), options.getBitLength(),
+        new Date(), 1);
+    cache.put(name, meta);
+    String versionName = buildVersionName(name, 0);
+    credentials.addSecretKey(nameT, meta.serialize());
+    credentials.addSecretKey(new Text(versionName), material);
+    return new KeyVersion(versionName, material);
+  }
+
+  @Override
+  public void deleteKey(String name) throws IOException {
+    Metadata meta = getMetadata(name);
+    if (meta == null) {
+      throw new IOException("Key " + name + " does not exist in " + this);
+    }
+    for(int v=0; v < meta.getVersions(); ++v) {
+      credentials.removeSecretKey(new Text(buildVersionName(name, v)));
+    }
+    credentials.removeSecretKey(new Text(name));
+    cache.remove(name);
+  }
+
+  @Override
+  public KeyVersion rollNewVersion(String name,
+                                    byte[] material) throws IOException {
+    Metadata meta = getMetadata(name);
+    if (meta == null) {
+      throw new IOException("Key " + name + " not found");
+    }
+    if (meta.getBitLength() != 8 * material.length) {
+      throw new IOException("Wrong key length. Required " +
+          meta.getBitLength() + ", but got " + (8 * material.length));
+    }
+    int nextVersion = meta.addVersion();
+    credentials.addSecretKey(new Text(name), meta.serialize());
+    String versionName = buildVersionName(name, nextVersion);
+    credentials.addSecretKey(new Text(versionName), material);
+    return new KeyVersion(versionName, material);
+  }
+
+  @Override
+  public String toString() {
+    return SCHEME_NAME + ":///";
+  }
+
+  @Override
+  public void flush() {
+    user.addCredentials(credentials);
+  }
+
+  public static class Factory extends KeyProviderFactory {
+
+    @Override
+    public KeyProvider createProvider(URI providerName,
+                                      Configuration conf) throws IOException {
+      if (SCHEME_NAME.equals(providerName.getScheme())) {
+        return new UserProvider();
+      }
+      return null;
+    }
+  }
+}

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java

@@ -294,6 +294,10 @@ public class HarFileSystem extends FileSystem {
   private String getHarAuth(URI underLyingUri) {
     String auth = underLyingUri.getScheme() + "-";
     if (underLyingUri.getHost() != null) {
+      if (underLyingUri.getUserInfo() != null) {
+        auth += underLyingUri.getUserInfo();
+        auth += "@";
+      }
       auth += underLyingUri.getHost();
       if (underLyingUri.getPort() != -1) {
         auth += ":";

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java

@@ -365,6 +365,10 @@ public class LocalDirAllocator {
           totalAvailable += availableOnDisk[i];
         }
 
+        if (totalAvailable == 0){
+          throw new DiskErrorException("No space available in any of the local directories.");
+        }
+
         // Keep rolling the wheel till we get a valid path
         Random r = new java.util.Random();
         while (numDirsSearched < numDirs && returnPath == null) {

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

@@ -305,12 +305,13 @@ public class HttpServer implements FilterContainer {
         }
       }
 
-      if (endpoints.size() == 0) {
+      if (endpoints.size() == 0 && connector == null) {
         throw new HadoopIllegalArgumentException("No endpoints specified");
       }
 
       if (hostName == null) {
-        hostName = endpoints.get(0).getHost();
+        hostName = endpoints.size() == 0 ? connector.getHost() : endpoints.get(
+            0).getHost();
       }
       
       if (this.conf == null) {

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

@@ -68,7 +68,7 @@ public class RetryPolicies {
    * </p>
    */
   public static final RetryPolicy RETRY_FOREVER = new RetryForever();
-  
+
   /**
    * <p>
    * Keep trying a limited number of times, waiting a fixed time between attempts,

+ 14 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcConstants.java

@@ -37,10 +37,24 @@ public class RpcConstants {
   
   public static final int INVALID_RETRY_COUNT = -1;
   
+ /**
+  * The Rpc-connection header is as follows 
+  * +----------------------------------+
+  * |  "hrpc" 4 bytes                  |      
+  * +----------------------------------+
+  * |  Version (1 byte)                |
+  * +----------------------------------+
+  * |  Service Class (1 byte)          |
+  * +----------------------------------+
+  * |  AuthProtocol (1 byte)           |      
+  * +----------------------------------+
+  */
+  
   /**
    * The first four bytes of Hadoop RPC connections
    */
   public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
+  public static final int HEADER_LEN_AFTER_HRPC_PART = 3; // 3 bytes that follow
   
   // 1 : Introduce ping and server does not throw away RPCs
   // 3 : Introduce the protocol into the RPC connection header

+ 127 - 44
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -551,14 +551,14 @@ public abstract class Server {
       
       @Override
       public void run() {
-        LOG.info("Starting " + getName());
+        LOG.info("Starting " + Thread.currentThread().getName());
         try {
           doRunLoop();
         } finally {
           try {
             readSelector.close();
           } catch (IOException ioe) {
-            LOG.error("Error closing read selector in " + this.getName(), ioe);
+            LOG.error("Error closing read selector in " + Thread.currentThread().getName(), ioe);
           }
         }
       }
@@ -589,7 +589,7 @@ public abstract class Server {
             }
           } catch (InterruptedException e) {
             if (running) {                      // unexpected -- log it
-              LOG.info(getName() + " unexpectedly interrupted", e);
+              LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
             }
           } catch (IOException ex) {
             LOG.error("Error in Reader", ex);
@@ -620,7 +620,7 @@ public abstract class Server {
 
     @Override
     public void run() {
-      LOG.info(getName() + ": starting");
+      LOG.info(Thread.currentThread().getName() + ": starting");
       SERVER.set(Server.this);
       connectionManager.startIdleScan();
       while (running) {
@@ -652,7 +652,7 @@ public abstract class Server {
           closeCurrentConnection(key, e);
         }
       }
-      LOG.info("Stopping " + this.getName());
+      LOG.info("Stopping " + Thread.currentThread().getName());
 
       synchronized (this) {
         try {
@@ -710,14 +710,14 @@ public abstract class Server {
       try {
         count = c.readAndProcess();
       } catch (InterruptedException ieo) {
-        LOG.info(getName() + ": readAndProcess caught InterruptedException", ieo);
+        LOG.info(Thread.currentThread().getName() + ": readAndProcess caught InterruptedException", ieo);
         throw ieo;
       } catch (Exception e) {
         // a WrappedRpcServerException is an exception that has been sent
         // to the client, so the stacktrace is unnecessary; any other
         // exceptions are unexpected internal server errors and thus the
         // stacktrace should be logged
-        LOG.info(getName() + ": readAndProcess from client " +
+        LOG.info(Thread.currentThread().getName() + ": readAndProcess from client " +
             c.getHostAddress() + " threw exception [" + e + "]",
             (e instanceof WrappedRpcServerException) ? null : e);
         count = -1; //so that the (count < 0) block is executed
@@ -740,7 +740,7 @@ public abstract class Server {
         try {
           acceptChannel.socket().close();
         } catch (IOException e) {
-          LOG.info(getName() + ":Exception in closing listener socket. " + e);
+          LOG.info(Thread.currentThread().getName() + ":Exception in closing listener socket. " + e);
         }
       }
       for (Reader r : readers) {
@@ -773,16 +773,16 @@ public abstract class Server {
 
     @Override
     public void run() {
-      LOG.info(getName() + ": starting");
+      LOG.info(Thread.currentThread().getName() + ": starting");
       SERVER.set(Server.this);
       try {
         doRunLoop();
       } finally {
-        LOG.info("Stopping " + this.getName());
+        LOG.info("Stopping " + Thread.currentThread().getName());
         try {
           writeSelector.close();
         } catch (IOException ioe) {
-          LOG.error("Couldn't close write selector in " + this.getName(), ioe);
+          LOG.error("Couldn't close write selector in " + Thread.currentThread().getName(), ioe);
         }
       }
     }
@@ -803,7 +803,7 @@ public abstract class Server {
                   doAsyncWrite(key);
               }
             } catch (IOException e) {
-              LOG.info(getName() + ": doAsyncWrite threw exception " + e);
+              LOG.info(Thread.currentThread().getName() + ": doAsyncWrite threw exception " + e);
             }
           }
           long now = Time.now();
@@ -918,7 +918,7 @@ public abstract class Server {
           call = responseQueue.removeFirst();
           SocketChannel channel = call.connection.channel;
           if (LOG.isDebugEnabled()) {
-            LOG.debug(getName() + ": responding to " + call);
+            LOG.debug(Thread.currentThread().getName() + ": responding to " + call);
           }
           //
           // Send as much data as we can in the non-blocking fashion
@@ -937,7 +937,7 @@ public abstract class Server {
               done = false;            // more calls pending to be sent.
             }
             if (LOG.isDebugEnabled()) {
-              LOG.debug(getName() + ": responding to " + call
+              LOG.debug(Thread.currentThread().getName() + ": responding to " + call
                   + " Wrote " + numBytes + " bytes.");
             }
           } else {
@@ -965,7 +965,7 @@ public abstract class Server {
               }
             }
             if (LOG.isDebugEnabled()) {
-              LOG.debug(getName() + ": responding to " + call
+              LOG.debug(Thread.currentThread().getName() + ": responding to " + call
                   + " Wrote partial " + numBytes + " bytes.");
             }
           }
@@ -973,7 +973,7 @@ public abstract class Server {
         }
       } finally {
         if (error && call != null) {
-          LOG.warn(getName()+", call " + call + ": output error");
+          LOG.warn(Thread.currentThread().getName()+", call " + call + ": output error");
           done = true;               // error. no more data for this channel.
           closeConnection(call.connection);
         }
@@ -1105,6 +1105,9 @@ public abstract class Server {
       this.channel = channel;
       this.lastContact = lastContact;
       this.data = null;
+      
+      // the buffer is initialized to read the "hrpc" and after that to read
+      // the length of the Rpc-packet (i.e 4 bytes)
       this.dataLengthBuffer = ByteBuffer.allocate(4);
       this.unwrappedData = null;
       this.unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
@@ -1200,7 +1203,16 @@ public abstract class Server {
       }
     }
 
-    private Throwable getCauseForInvalidToken(IOException e) {
+    /**
+     * Some exceptions ({@link RetriableException} and {@link StandbyException})
+     * that are wrapped as a cause of parameter e are unwrapped so that they can
+     * be sent as the true cause to the client side. In case of
+     * {@link InvalidToken} we go one level deeper to get the true cause.
+     * 
+     * @param e the exception that may have a cause we want to unwrap.
+     * @return the true cause for some exceptions.
+     */
+    private Throwable getTrueCause(IOException e) {
       Throwable cause = e;
       while (cause != null) {
         if (cause instanceof RetriableException) {
@@ -1223,6 +1235,18 @@ public abstract class Server {
       return e;
     }
     
+    /**
+     * Process saslMessage and send saslResponse back
+     * @param saslMessage received SASL message
+     * @throws WrappedRpcServerException setup failed due to SASL negotiation 
+     *         failure, premature or invalid connection context, or other state 
+     *         errors. This exception needs to be sent to the client. This 
+     *         exception will wrap {@link RetriableException}, 
+     *         {@link InvalidToken}, {@link StandbyException} or 
+     *         {@link SaslException}.
+     * @throws IOException if sending reply fails
+     * @throws InterruptedException
+     */
     private void saslProcess(RpcSaslProto saslMessage)
         throws WrappedRpcServerException, IOException, InterruptedException {
       if (saslContextEstablished) {
@@ -1239,7 +1263,7 @@ public abstract class Server {
           // attempting user could be null
           AUDITLOG.warn(AUTH_FAILED_FOR + this.toString() + ":"
               + attemptingUser + " (" + e.getLocalizedMessage() + ")");
-          throw (IOException) getCauseForInvalidToken(e);
+          throw (IOException) getTrueCause(e);
         }
         
         if (saslServer != null && saslServer.isComplete()) {
@@ -1274,13 +1298,26 @@ public abstract class Server {
       }
     }
     
+    /**
+     * Process a saslMessge.
+     * @param saslMessage received SASL message
+     * @return the sasl response to send back to client
+     * @throws SaslException if authentication or generating response fails, 
+     *                       or SASL protocol mixup
+     * @throws IOException if a SaslServer cannot be created
+     * @throws AccessControlException if the requested authentication type 
+     *         is not supported or trying to re-attempt negotiation.
+     * @throws InterruptedException
+     */
     private RpcSaslProto processSaslMessage(RpcSaslProto saslMessage)
-        throws IOException, InterruptedException {
+        throws SaslException, IOException, AccessControlException,
+        InterruptedException {
       RpcSaslProto saslResponse = null;
       final SaslState state = saslMessage.getState(); // required      
       switch (state) {
         case NEGOTIATE: {
           if (sentNegotiate) {
+            // FIXME shouldn't this be SaslException?
             throw new AccessControlException(
                 "Client already attempted negotiation");
           }
@@ -1402,12 +1439,30 @@ public abstract class Server {
       }
     }
 
+    /**
+     * This method reads in a non-blocking fashion from the channel: 
+     * this method is called repeatedly when data is present in the channel; 
+     * when it has enough data to process one rpc it processes that rpc.
+     * 
+     * On the first pass, it processes the connectionHeader, 
+     * connectionContext (an outOfBand RPC) and at most one RPC request that 
+     * follows that. On future passes it will process at most one RPC request.
+     *  
+     * Quirky things: dataLengthBuffer (4 bytes) is used to read "hrpc" OR 
+     * rpc request length.
+     *    
+     * @return -1 in case of error, else num bytes read so far
+     * @throws WrappedRpcServerException - an exception that has already been 
+     *         sent back to the client that does not require verbose logging
+     *         by the Listener thread
+     * @throws IOException - internal error that should not be returned to
+     *         client, typically failure to respond to client
+     * @throws InterruptedException
+     */
     public int readAndProcess()
         throws WrappedRpcServerException, IOException, InterruptedException {
       while (true) {
-        /* Read at most one RPC. If the header is not read completely yet
-         * then iterate until we read first RPC or until there is no data left.
-         */    
+        // dataLengthBuffer is used to read "hrpc" or the rpc-packet length
         int count = -1;
         if (dataLengthBuffer.remaining() > 0) {
           count = channelRead(channel, dataLengthBuffer);       
@@ -1416,9 +1471,11 @@ public abstract class Server {
         }
         
         if (!connectionHeaderRead) {
-          //Every connection is expected to send the header.
+          // Every connection is expected to send the header;
+          // so far we read "hrpc" of the connection header.
           if (connectionHeaderBuf == null) {
-            connectionHeaderBuf = ByteBuffer.allocate(3);
+            // for the bytes that follow "hrpc", in the connection header
+            connectionHeaderBuf = ByteBuffer.allocate(HEADER_LEN_AFTER_HRPC_PART);
           }
           count = channelRead(channel, connectionHeaderBuf);
           if (count < 0 || connectionHeaderBuf.remaining() > 0) {
@@ -1451,27 +1508,30 @@ public abstract class Server {
           // this may switch us into SIMPLE
           authProtocol = initializeAuthContext(connectionHeaderBuf.get(2));          
           
-          dataLengthBuffer.clear();
+          dataLengthBuffer.clear(); // clear to next read rpc packet len
           connectionHeaderBuf = null;
           connectionHeaderRead = true;
-          continue;
+          continue; // connection header read, now read  4 bytes rpc packet len
         }
         
-        if (data == null) {
+        if (data == null) { // just read 4 bytes -  length of RPC packet
           dataLengthBuffer.flip();
           dataLength = dataLengthBuffer.getInt();
           checkDataLength(dataLength);
+          // Set buffer for reading EXACTLY the RPC-packet length and no more.
           data = ByteBuffer.allocate(dataLength);
         }
-        
+        // Now read the RPC packet
         count = channelRead(channel, data);
         
         if (data.remaining() == 0) {
-          dataLengthBuffer.clear();
+          dataLengthBuffer.clear(); // to read length of future rpc packets
           data.flip();
           boolean isHeaderRead = connectionContextRead;
           processOneRpc(data.array());
           data = null;
+          // the last rpc-request we processed could have simply been the
+          // connectionContext; if so continue to read the first RPC.
           if (!isHeaderRead) {
             continue;
           }
@@ -1508,8 +1568,16 @@ public abstract class Server {
       return authProtocol;
     }
 
+    /**
+     * Process the Sasl's Negotiate request, including the optimization of 
+     * accelerating token negotiation.
+     * @return the response to Negotiate request - the list of enabled 
+     *         authMethods and challenge if the TOKENS are supported. 
+     * @throws SaslException - if attempt to generate challenge fails.
+     * @throws IOException - if it fails to create the SASL server for Tokens
+     */
     private RpcSaslProto buildSaslNegotiateResponse()
-        throws IOException, InterruptedException {
+        throws InterruptedException, SaslException, IOException {
       RpcSaslProto negotiateMessage = negotiateResponse;
       // accelerate token negotiation by sending initial challenge
       // in the negotiation response
@@ -1635,8 +1703,11 @@ public abstract class Server {
     /**
      * Process a wrapped RPC Request - unwrap the SASL packet and process
      * each embedded RPC request 
-     * @param buf - SASL wrapped request of one or more RPCs
+     * @param inBuf - SASL wrapped request of one or more RPCs
      * @throws IOException - SASL packet cannot be unwrapped
+     * @throws WrappedRpcServerException - an exception that has already been 
+     *         sent back to the client that does not require verbose logging
+     *         by the Listener thread
      * @throws InterruptedException
      */    
     private void unwrapPacketAndProcessRpcs(byte[] inBuf)
@@ -1677,13 +1748,21 @@ public abstract class Server {
     }
     
     /**
-     * Process an RPC Request - handle connection setup and decoding of
-     * request into a Call
+     * Process one RPC Request from buffer read from socket stream 
+     *  - decode rpc in a rpc-Call
+     *  - handle out-of-band RPC requests such as the initial connectionContext
+     *  - A successfully decoded RpcCall will be deposited in RPC-Q and
+     *    its response will be sent later when the request is processed.
+     * 
+     * Prior to this call the connectionHeader ("hrpc...") has been handled and
+     * if SASL then SASL has been established and the buf we are passed
+     * has been unwrapped from SASL.
+     * 
      * @param buf - contains the RPC request header and the rpc request
      * @throws IOException - internal error that should not be returned to
      *         client, typically failure to respond to client
-     * @throws WrappedRpcServerException - an exception to be sent back to
-     *         the client that does not require verbose logging by the
+     * @throws WrappedRpcServerException - an exception that is sent back to the
+     *         client in this method and does not require verbose logging by the
      *         Listener thread
      * @throws InterruptedException
      */    
@@ -1753,8 +1832,11 @@ public abstract class Server {
     }
 
     /**
-     * Process an RPC Request - the connection headers and context must
-     * have been already read
+     * Process an RPC Request 
+     *   - the connection headers and context must have been already read.
+     *   - Based on the rpcKind, decode the rpcRequest.
+     *   - A successfully decoded RpcCall will be deposited in RPC-Q and
+     *     its response will be sent later when the request is processed.
      * @param header - RPC request header
      * @param dis - stream to request payload
      * @throws WrappedRpcServerException - due to fatal rpc layer issues such
@@ -1803,7 +1885,8 @@ public abstract class Server {
      * @param dis - stream to request payload
      * @throws WrappedRpcServerException - setup failed due to SASL
      *         negotiation failure, premature or invalid connection context,
-     *         or other state errors 
+     *         or other state errors. This exception needs to be sent to the 
+     *         client.
      * @throws IOException - failed to send a response back to the client
      * @throws InterruptedException
      */
@@ -1928,7 +2011,7 @@ public abstract class Server {
 
     @Override
     public void run() {
-      LOG.debug(getName() + ": starting");
+      LOG.debug(Thread.currentThread().getName() + ": starting");
       SERVER.set(Server.this);
       ByteArrayOutputStream buf = 
         new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE);
@@ -1936,7 +2019,7 @@ public abstract class Server {
         try {
           final Call call = callQueue.take(); // pop the queue; maybe blocked here
           if (LOG.isDebugEnabled()) {
-            LOG.debug(getName() + ": " + call + " for RpcKind " + call.rpcKind);
+            LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
           }
           String errorClass = null;
           String error = null;
@@ -1969,7 +2052,7 @@ public abstract class Server {
             if (e instanceof UndeclaredThrowableException) {
               e = e.getCause();
             }
-            String logMsg = getName() + ", call " + call + ": error: " + e;
+            String logMsg = Thread.currentThread().getName() + ", call " + call + ": error: " + e;
             if (e instanceof RuntimeException || e instanceof Error) {
               // These exception types indicate something is probably wrong
               // on the server side, as opposed to just a normal exceptional
@@ -2018,13 +2101,13 @@ public abstract class Server {
           }
         } catch (InterruptedException e) {
           if (running) {                          // unexpected -- log it
-            LOG.info(getName() + " unexpectedly interrupted", e);
+            LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
           }
         } catch (Exception e) {
-          LOG.info(getName() + " caught an exception", e);
+          LOG.info(Thread.currentThread().getName() + " caught an exception", e);
         }
       }
-      LOG.debug(getName() + ": exiting");
+      LOG.debug(Thread.currentThread().getName() + ": exiting");
     }
 
   }

+ 11 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java

@@ -24,10 +24,8 @@ import java.lang.management.MemoryUsage;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
 import java.lang.management.GarbageCollectorMXBean;
-import java.util.Map;
 import java.util.List;
-
-import com.google.common.collect.Maps;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.log.metrics.EventCounter;
@@ -67,7 +65,8 @@ public class JvmMetrics implements MetricsSource {
       ManagementFactory.getGarbageCollectorMXBeans();
   final ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
   final String processName, sessionId;
-  final Map<String, MetricsInfo[]> gcInfoCache = Maps.newHashMap();
+  final ConcurrentHashMap<String, MetricsInfo[]> gcInfoCache =
+      new ConcurrentHashMap<String, MetricsInfo[]>();
 
   JvmMetrics(String processName, String sessionId) {
     this.processName = processName;
@@ -123,13 +122,17 @@ public class JvmMetrics implements MetricsSource {
       .addCounter(GcTimeMillis, timeMillis);
   }
 
-  private synchronized MetricsInfo[] getGcInfo(String gcName) {
+  private MetricsInfo[] getGcInfo(String gcName) {
     MetricsInfo[] gcInfo = gcInfoCache.get(gcName);
     if (gcInfo == null) {
       gcInfo = new MetricsInfo[2];
-      gcInfo[0] = Interns.info("GcCount"+ gcName, "GC Count for "+ gcName);
-      gcInfo[1] = Interns.info("GcTimeMillis"+ gcName, "GC Time for "+ gcName);
-      gcInfoCache.put(gcName, gcInfo);
+      gcInfo[0] = Interns.info("GcCount" + gcName, "GC Count for " + gcName);
+      gcInfo[1] = Interns
+          .info("GcTimeMillis" + gcName, "GC Time for " + gcName);
+      MetricsInfo[] previousGcInfo = gcInfoCache.putIfAbsent(gcName, gcInfo);
+      if (previousGcInfo != null) {
+        return previousGcInfo;
+      }
     }
     return gcInfo;
   }

+ 18 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocket.java

@@ -276,6 +276,24 @@ public class DomainSocket implements Closeable {
     return new DomainSocket(path, fd);
   }
 
+  /**
+   * Create a pair of UNIX domain sockets which are connected to each other
+   * by calling socketpair(2).
+   *
+   * @return                An array of two UNIX domain sockets connected to
+   *                        each other.
+   * @throws IOException    on error.
+   */
+  public static DomainSocket[] socketpair() throws IOException {
+    int fds[] = socketpair0();
+    return new DomainSocket[] {
+      new DomainSocket("(anonymous0)", fds[0]),
+      new DomainSocket("(anonymous1)", fds[1])
+    };
+  }
+
+  private static native int[] socketpair0() throws IOException;
+
   private static native int accept0(int fd) throws IOException;
 
   /**

+ 9 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java

@@ -133,7 +133,15 @@ public class Credentials implements Writable {
   public void addSecretKey(Text alias, byte[] key) {
     secretKeysMap.put(alias, key);
   }
- 
+
+  /**
+   * Remove the key for a given alias.
+   * @param alias the alias for the key
+   */
+  public void removeSecretKey(Text alias) {
+    secretKeysMap.remove(alias);
+  }
+
   /**
    * Convenience method for reading a token storage file, and loading the Tokens
    * therein in the passed UGI

+ 60 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java

@@ -25,6 +25,10 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.security.Security;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
@@ -38,6 +42,7 @@ import javax.security.sasl.RealmCallback;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.logging.Log;
@@ -63,6 +68,7 @@ public class SaslRpcServer {
   public static final String SASL_DEFAULT_REALM = "default";
   public static final Map<String, String> SASL_PROPS = 
       new TreeMap<String, String>();
+  private static SaslServerFactory saslFactory;
 
   public static enum QualityOfProtection {
     AUTHENTICATION("auth"),
@@ -125,7 +131,7 @@ public class SaslRpcServer {
   public SaslServer create(Connection connection,
                            SecretManager<TokenIdentifier> secretManager
       ) throws IOException, InterruptedException {
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    UserGroupInformation ugi = null;
     final CallbackHandler callback;
     switch (authMethod) {
       case TOKEN: {
@@ -133,6 +139,7 @@ public class SaslRpcServer {
         break;
       }
       case KERBEROS: {
+        ugi = UserGroupInformation.getCurrentUser();
         if (serverId.isEmpty()) {
           throw new AccessControlException(
               "Kerberos principal name does NOT have the expected "
@@ -147,14 +154,20 @@ public class SaslRpcServer {
             "Server does not support SASL " + authMethod);
     }
     
-    SaslServer saslServer = ugi.doAs(
+    final SaslServer saslServer;
+    if (ugi != null) {
+      saslServer = ugi.doAs(
         new PrivilegedExceptionAction<SaslServer>() {
           @Override
           public SaslServer run() throws SaslException  {
-            return Sasl.createSaslServer(mechanism, protocol, serverId,
+            return saslFactory.createSaslServer(mechanism, protocol, serverId,
                 SaslRpcServer.SASL_PROPS, callback);
           }
         });
+    } else {
+      saslServer = saslFactory.createSaslServer(mechanism, protocol, serverId,
+          SaslRpcServer.SASL_PROPS, callback);
+    }
     if (saslServer == null) {
       throw new AccessControlException(
           "Unable to find SASL server implementation for " + mechanism);
@@ -180,6 +193,7 @@ public class SaslRpcServer {
     SASL_PROPS.put(Sasl.QOP, saslQOP.getSaslQop());
     SASL_PROPS.put(Sasl.SERVER_AUTH, "true");
     Security.addProvider(new SaslPlainServer.SecurityProvider());
+    saslFactory = new FastSaslServerFactory(SASL_PROPS);
   }
   
   static String encodeIdentifier(byte[] identifier) {
@@ -363,4 +377,47 @@ public class SaslRpcServer {
       }
     }
   }
+  
+  // Sasl.createSaslServer is 100-200X slower than caching the factories!
+  private static class FastSaslServerFactory implements SaslServerFactory {
+    private final Map<String,List<SaslServerFactory>> factoryCache =
+        new HashMap<String,List<SaslServerFactory>>();
+
+    FastSaslServerFactory(Map<String,?> props) {
+      final Enumeration<SaslServerFactory> factories =
+          Sasl.getSaslServerFactories();
+      while (factories.hasMoreElements()) {
+        SaslServerFactory factory = factories.nextElement();
+        for (String mech : factory.getMechanismNames(props)) {
+          if (!factoryCache.containsKey(mech)) {
+            factoryCache.put(mech, new ArrayList<SaslServerFactory>());
+          }
+          factoryCache.get(mech).add(factory);
+        }
+      }
+    }
+
+    @Override
+    public SaslServer createSaslServer(String mechanism, String protocol,
+        String serverName, Map<String,?> props, CallbackHandler cbh)
+        throws SaslException {
+      SaslServer saslServer = null;
+      List<SaslServerFactory> factories = factoryCache.get(mechanism);
+      if (factories != null) {
+        for (SaslServerFactory factory : factories) {
+          saslServer = factory.createSaslServer(
+              mechanism, protocol, serverName, props, cbh);
+          if (saslServer != null) {
+            break;
+          }
+        }
+      }
+      return saslServer;
+    }
+
+    @Override
+    public String[] getMechanismNames(Map<String, ?> props) {
+      return factoryCache.keySet().toArray(new String[0]);
+    }
+  }
 }

+ 51 - 36
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -477,7 +477,7 @@ public class UserGroupInformation {
     
     private static final AppConfigurationEntry[] SIMPLE_CONF = 
       new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN, HADOOP_LOGIN};
-
+    
     private static final AppConfigurationEntry[] USER_KERBEROS_CONF =
       new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN, USER_KERBEROS_LOGIN,
                                   HADOOP_LOGIN};
@@ -682,44 +682,59 @@ public class UserGroupInformation {
   public synchronized 
   static UserGroupInformation getLoginUser() throws IOException {
     if (loginUser == null) {
-      ensureInitialized();
-      try {
-        Subject subject = new Subject();
-        LoginContext login =
-            newLoginContext(authenticationMethod.getLoginAppName(), 
-                            subject, new HadoopConfiguration());
-        login.login();
-        UserGroupInformation realUser = new UserGroupInformation(subject);
-        realUser.setLogin(login);
-        realUser.setAuthenticationMethod(authenticationMethod);
-        realUser = new UserGroupInformation(login.getSubject());
-        // If the HADOOP_PROXY_USER environment variable or property
-        // is specified, create a proxy user as the logged in user.
-        String proxyUser = System.getenv(HADOOP_PROXY_USER);
-        if (proxyUser == null) {
-          proxyUser = System.getProperty(HADOOP_PROXY_USER);
-        }
-        loginUser = proxyUser == null ? realUser : createProxyUser(proxyUser, realUser);
-
-        String fileLocation = System.getenv(HADOOP_TOKEN_FILE_LOCATION);
-        if (fileLocation != null) {
-          // Load the token storage file and put all of the tokens into the
-          // user. Don't use the FileSystem API for reading since it has a lock
-          // cycle (HADOOP-9212).
-          Credentials cred = Credentials.readTokenStorageFile(
-              new File(fileLocation), conf);
-          loginUser.addCredentials(cred);
-        }
-        loginUser.spawnAutoRenewalThreadForUserCreds();
-      } catch (LoginException le) {
-        LOG.debug("failure to login", le);
-        throw new IOException("failure to login", le);
+      loginUserFromSubject(null);
+    }
+    return loginUser;
+  }
+  
+  /**
+   * Log in a user using the given subject
+   * @parma subject the subject to use when logging in a user, or null to 
+   * create a new subject.
+   * @throws IOException if login fails
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public synchronized 
+  static void loginUserFromSubject(Subject subject) throws IOException {
+    ensureInitialized();
+    try {
+      if (subject == null) {
+        subject = new Subject();
       }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("UGI loginUser:"+loginUser);
+      LoginContext login =
+          newLoginContext(authenticationMethod.getLoginAppName(), 
+                          subject, new HadoopConfiguration());
+      login.login();
+      UserGroupInformation realUser = new UserGroupInformation(subject);
+      realUser.setLogin(login);
+      realUser.setAuthenticationMethod(authenticationMethod);
+      realUser = new UserGroupInformation(login.getSubject());
+      // If the HADOOP_PROXY_USER environment variable or property
+      // is specified, create a proxy user as the logged in user.
+      String proxyUser = System.getenv(HADOOP_PROXY_USER);
+      if (proxyUser == null) {
+        proxyUser = System.getProperty(HADOOP_PROXY_USER);
+      }
+      loginUser = proxyUser == null ? realUser : createProxyUser(proxyUser, realUser);
+
+      String fileLocation = System.getenv(HADOOP_TOKEN_FILE_LOCATION);
+      if (fileLocation != null) {
+        // Load the token storage file and put all of the tokens into the
+        // user. Don't use the FileSystem API for reading since it has a lock
+        // cycle (HADOOP-9212).
+        Credentials cred = Credentials.readTokenStorageFile(
+            new File(fileLocation), conf);
+        loginUser.addCredentials(cred);
       }
+      loginUser.spawnAutoRenewalThreadForUserCreds();
+    } catch (LoginException le) {
+      LOG.debug("failure to login", le);
+      throw new IOException("failure to login", le);
     }
-    return loginUser;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("UGI loginUser:"+loginUser);
+    } 
   }
 
   @InterfaceAudience.Private

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java

@@ -275,8 +275,9 @@ public class ReflectionUtils {
   
   /**
    * Make a copy of the writable object using serialization to a buffer
-   * @param dst the object to copy from
-   * @param src the object to copy into, which is destroyed
+   * @param src the object to copy from
+   * @param dst the object to copy into, which is destroyed
+   * @return dst param (the copy)
    * @throws IOException
    */
   @SuppressWarnings("unchecked")

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java

@@ -928,8 +928,10 @@ public class StringUtils {
    * @param args  List of arguments.
    * @return      null if the option was not found; the value of the 
    *              option otherwise.
+   * @throws IllegalArgumentException if the option's argument is not present
    */
-  public static String popOptionWithArgument(String name, List<String> args) {
+  public static String popOptionWithArgument(String name, List<String> args)
+      throws IllegalArgumentException {
     String val = null;
     for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
       String cur = iter.next();
@@ -939,7 +941,7 @@ public class StringUtils {
       } else if (cur.equals(name)) {
         iter.remove();
         if (!iter.hasNext()) {
-          throw new RuntimeException("option " + name + " requires 1 " +
+          throw new IllegalArgumentException("option " + name + " requires 1 " +
               "argument.");
         }
         val = iter.next();

+ 44 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/net/unix/DomainSocket.c

@@ -364,6 +364,50 @@ JNIEnv *env, jclass clazz, jstring path)
   return fd;
 }
 
+#define SOCKETPAIR_ARRAY_LEN 2
+
+JNIEXPORT jarray JNICALL
+Java_org_apache_hadoop_net_unix_DomainSocket_socketpair0(
+JNIEnv *env, jclass clazz)
+{
+  jarray arr = NULL;
+  int idx, err, fds[SOCKETPAIR_ARRAY_LEN] = { -1, -1 };
+  jthrowable jthr = NULL;
+
+  arr = (*env)->NewIntArray(env, SOCKETPAIR_ARRAY_LEN);
+  jthr = (*env)->ExceptionOccurred(env);
+  if (jthr) {
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  if (socketpair(PF_UNIX, SOCK_STREAM, 0, fds) < 0) {
+    err = errno;
+    jthr = newSocketException(env, err,
+            "socketpair(2) error: %s", terror(err));
+    goto done;
+  }
+  (*env)->SetIntArrayRegion(env, arr, 0, SOCKETPAIR_ARRAY_LEN, fds);
+  jthr = (*env)->ExceptionOccurred(env);
+  if (jthr) {
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+
+done:
+  if (jthr) {
+    (*env)->DeleteLocalRef(env, arr);
+    arr = NULL;
+    for (idx = 0; idx < SOCKETPAIR_ARRAY_LEN; idx++) {
+      if (fds[idx] >= 0) {
+        close(fds[idx]);
+        fds[idx] = -1;
+      }
+    }
+    (*env)->Throw(env, jthr);
+  }
+  return arr;
+}
+
 JNIEXPORT jint JNICALL
 Java_org_apache_hadoop_net_unix_DomainSocket_accept0(
 JNIEnv *env, jclass clazz, jint fd)

+ 41 - 6
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_user_info.c

@@ -122,13 +122,43 @@ int hadoop_user_info_fetch(struct hadoop_user_info *uinfo,
   }
 }
 
+static int put_primary_gid_first(struct hadoop_user_info *uinfo)
+{
+  int i, num_gids = uinfo->num_gids;
+  gid_t first_gid;
+  gid_t gid;
+  gid_t primary = uinfo->pwd.pw_gid;
+
+  if (num_gids < 1) {
+    // There are no gids, but we expected at least one.
+    return EINVAL;
+  }
+  first_gid = uinfo->gids[0];
+  if (first_gid == primary) {
+    // First gid is already the primary.
+    return 0;
+  }
+  for (i = 1; i < num_gids; i++) {
+    gid = uinfo->gids[i];
+    if (gid == primary) {
+      // swap first gid and this gid.
+      uinfo->gids[0] = gid;
+      uinfo->gids[i] = first_gid;
+      return 0;
+    }
+  }
+  // Did not find the primary gid in the list.
+  return EINVAL;
+}
+
 int hadoop_user_info_getgroups(struct hadoop_user_info *uinfo)
 {
   int ret, ngroups;
   gid_t *ngids;
 
   if (!uinfo->pwd.pw_name) {
-    return EINVAL; // invalid user info
+    // invalid user info
+    return EINVAL;
   }
   uinfo->num_gids = 0;
   if (!uinfo->gids) {
@@ -141,8 +171,12 @@ int hadoop_user_info_getgroups(struct hadoop_user_info *uinfo)
   ngroups = uinfo->gids_size;
   ret = getgrouplist(uinfo->pwd.pw_name, uinfo->pwd.pw_gid, 
                          uinfo->gids, &ngroups);
-  if (ret != -1) {
+  if (ret > 0) {
     uinfo->num_gids = ngroups;
+    ret = put_primary_gid_first(uinfo);
+    if (ret) {
+      return ret;
+    }
     return 0;
   }
   ngids = realloc(uinfo->gids, sizeof(uinfo->gids[0]) * ngroups);
@@ -153,11 +187,12 @@ int hadoop_user_info_getgroups(struct hadoop_user_info *uinfo)
   uinfo->gids_size = ngroups;
   ret = getgrouplist(uinfo->pwd.pw_name, uinfo->pwd.pw_gid, 
                          uinfo->gids, &ngroups);
-  if (ret != -1) {
-    uinfo->num_gids = ngroups;
-    return 0;
+  if (ret < 0) {
+    return EIO;
   }
-  return EIO;
+  uinfo->num_gids = ngroups;
+  ret = put_primary_gid_first(uinfo);
+  return ret;
 }
 
 #ifdef USER_TESTING

+ 17 - 0
hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory

@@ -0,0 +1,17 @@
+# 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.
+
+org.apache.hadoop.crypto.key.JavaKeyStoreProvider$Factory
+org.apache.hadoop.crypto.key.UserProvider$Factory

+ 112 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProvider.java

@@ -0,0 +1,112 @@
+/**
+ * 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.crypto.key;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestKeyProvider {
+
+  @Test
+  public void testBuildVersionName() throws Exception {
+    assertEquals("/a/b@3", KeyProvider.buildVersionName("/a/b", 3));
+    assertEquals("/aaa@12", KeyProvider.buildVersionName("/aaa", 12));
+  }
+
+  @Test
+  public void testParseVersionName() throws Exception {
+    assertEquals("/a/b", KeyProvider.getBaseName("/a/b@3"));
+    assertEquals("/aaa", KeyProvider.getBaseName("/aaa@112"));
+    try {
+      KeyProvider.getBaseName("no-slashes");
+      assertTrue("should have thrown", false);
+    } catch (IOException e) {
+      assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testKeyMaterial() throws Exception {
+    byte[] key1 = new byte[]{1,2,3,4};
+    KeyProvider.KeyVersion obj = new KeyProvider.KeyVersion("key1@1", key1);
+    assertEquals("key1@1", obj.getVersionName());
+    assertArrayEquals(new byte[]{1,2,3,4}, obj.getMaterial());
+  }
+
+  @Test
+  public void testMetadata() throws Exception {
+    DateFormat format = new SimpleDateFormat("y/m/d");
+    Date date = format.parse("2013/12/25");
+    KeyProvider.Metadata meta = new KeyProvider.Metadata("myCipher", 100,
+        date, 123);
+    assertEquals("myCipher", meta.getCipher());
+    assertEquals(100, meta.getBitLength());
+    assertEquals(date, meta.getCreated());
+    assertEquals(123, meta.getVersions());
+    KeyProvider.Metadata second = new KeyProvider.Metadata(meta.serialize());
+    assertEquals(meta.getCipher(), second.getCipher());
+    assertEquals(meta.getBitLength(), second.getBitLength());
+    assertEquals(meta.getCreated(), second.getCreated());
+    assertEquals(meta.getVersions(), second.getVersions());
+    int newVersion = second.addVersion();
+    assertEquals(123, newVersion);
+    assertEquals(124, second.getVersions());
+    assertEquals(123, meta.getVersions());
+  }
+
+  @Test
+  public void testOptions() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(KeyProvider.DEFAULT_CIPHER_NAME, "myCipher");
+    conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 512);
+    KeyProvider.Options options = KeyProvider.options(conf);
+    assertEquals("myCipher", options.getCipher());
+    assertEquals(512, options.getBitLength());
+    options.setCipher("yourCipher");
+    options.setBitLength(128);
+    assertEquals("yourCipher", options.getCipher());
+    assertEquals(128, options.getBitLength());
+    options = KeyProvider.options(new Configuration());
+    assertEquals(KeyProvider.DEFAULT_CIPHER, options.getCipher());
+    assertEquals(KeyProvider.DEFAULT_BITLENGTH, options.getBitLength());
+  }
+
+  @Test
+  public void testUnnestUri() throws Exception {
+    assertEquals(new Path("hdfs://nn.example.com/my/path"),
+        KeyProvider.unnestUri(new URI("myscheme://hdfs@nn.example.com/my/path")));
+    assertEquals(new Path("hdfs://nn/my/path?foo=bar&baz=bat#yyy"),
+        KeyProvider.unnestUri(new URI("myscheme://hdfs@nn/my/path?foo=bar&baz=bat#yyy")));
+    assertEquals(new Path("inner://hdfs@nn1.example.com/my/path"),
+        KeyProvider.unnestUri(new URI("outer://inner@hdfs@nn1.example.com/my/path")));
+    assertEquals(new Path("user:///"),
+        KeyProvider.unnestUri(new URI("outer://user/")));
+  }
+}

+ 191 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java

@@ -0,0 +1,191 @@
+/**
+ * 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.crypto.key;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestKeyProviderFactory {
+
+  private static final File tmpDir =
+      new File(System.getProperty("test.build.data", "/tmp"), "key");
+
+  @Test
+  public void testFactory() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
+        UserProvider.SCHEME_NAME + ":///," +
+            JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks");
+    List<KeyProvider> providers = KeyProviderFactory.getProviders(conf);
+    assertEquals(2, providers.size());
+    assertEquals(UserProvider.class, providers.get(0).getClass());
+    assertEquals(JavaKeyStoreProvider.class, providers.get(1).getClass());
+    assertEquals(UserProvider.SCHEME_NAME +
+        ":///", providers.get(0).toString());
+    assertEquals(JavaKeyStoreProvider.SCHEME_NAME +
+        "://file" + tmpDir + "/test.jks",
+        providers.get(1).toString());
+  }
+
+  @Test
+  public void testFactoryErrors() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, "unknown:///");
+    try {
+      List<KeyProvider> providers = KeyProviderFactory.getProviders(conf);
+      assertTrue("should throw!", false);
+    } catch (IOException e) {
+      assertEquals("No KeyProviderFactory for unknown:/// in " +
+          KeyProviderFactory.KEY_PROVIDER_PATH,
+          e.getMessage());
+    }
+  }
+
+  @Test
+  public void testUriErrors() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, "unkn@own:/x/y");
+    try {
+      List<KeyProvider> providers = KeyProviderFactory.getProviders(conf);
+      assertTrue("should throw!", false);
+    } catch (IOException e) {
+      assertEquals("Bad configuration of " +
+          KeyProviderFactory.KEY_PROVIDER_PATH +
+          " at unkn@own:/x/y", e.getMessage());
+    }
+  }
+
+  static void checkSpecificProvider(Configuration conf,
+                                   String ourUrl) throws Exception {
+    KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
+    byte[] key1 = new byte[32];
+    byte[] key2 = new byte[32];
+    byte[] key3 = new byte[32];
+    for(int i =0; i < key1.length; ++i) {
+      key1[i] = (byte) i;
+      key2[i] = (byte) (i * 2);
+      key3[i] = (byte) (i * 3);
+    }
+    // ensure that we get nulls when the key isn't there
+    assertEquals(null, provider.getKeyVersion("no-such-key"));
+    assertEquals(null, provider.getMetadata("key"));
+    // create a new key
+    try {
+      provider.createKey("key3", key3, KeyProvider.options(conf));
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+    // check the metadata for key3
+    KeyProvider.Metadata meta = provider.getMetadata("key3");
+    assertEquals(KeyProvider.DEFAULT_CIPHER, meta.getCipher());
+    assertEquals(KeyProvider.DEFAULT_BITLENGTH, meta.getBitLength());
+    assertEquals(1, meta.getVersions());
+    // make sure we get back the right key
+    assertArrayEquals(key3, provider.getCurrentKey("key3").getMaterial());
+    assertEquals("key3@0", provider.getCurrentKey("key3").getVersionName());
+    // try recreating key3
+    try {
+      provider.createKey("key3", key3, KeyProvider.options(conf));
+      assertTrue("should throw", false);
+    } catch (IOException e) {
+      assertEquals("Key key3 already exists in " + ourUrl, e.getMessage());
+    }
+    provider.deleteKey("key3");
+    try {
+      provider.deleteKey("key3");
+      assertTrue("should throw", false);
+    } catch (IOException e) {
+      assertEquals("Key key3 does not exist in " + ourUrl, e.getMessage());
+    }
+    provider.createKey("key3", key3, KeyProvider.options(conf));
+    try {
+      provider.createKey("key4", key3,
+          KeyProvider.options(conf).setBitLength(8));
+      assertTrue("should throw", false);
+    } catch (IOException e) {
+      assertEquals("Wrong key length. Required 8, but got 256", e.getMessage());
+    }
+    provider.createKey("key4", new byte[]{1},
+        KeyProvider.options(conf).setBitLength(8));
+    provider.rollNewVersion("key4", new byte[]{2});
+    meta = provider.getMetadata("key4");
+    assertEquals(2, meta.getVersions());
+    assertArrayEquals(new byte[]{2},
+        provider.getCurrentKey("key4").getMaterial());
+    assertArrayEquals(new byte[]{1},
+        provider.getKeyVersion("key4@0").getMaterial());
+    assertEquals("key4@1", provider.getCurrentKey("key4").getVersionName());
+    try {
+      provider.rollNewVersion("key4", key1);
+      assertTrue("should throw", false);
+    } catch (IOException e) {
+      assertEquals("Wrong key length. Required 8, but got 256", e.getMessage());
+    }
+    try {
+      provider.rollNewVersion("no-such-key", key1);
+      assertTrue("should throw", false);
+    } catch (IOException e) {
+      assertEquals("Key no-such-key not found", e.getMessage());
+    }
+    provider.flush();
+    // get a new instance of the provider to ensure it was saved correctly
+    provider = KeyProviderFactory.getProviders(conf).get(0);
+    assertArrayEquals(new byte[]{2},
+        provider.getCurrentKey("key4").getMaterial());
+    assertArrayEquals(key3, provider.getCurrentKey("key3").getMaterial());
+    assertEquals("key3@0", provider.getCurrentKey("key3").getVersionName());
+  }
+
+  @Test
+  public void testUserProvider() throws Exception {
+    Configuration conf = new Configuration();
+    final String ourUrl = UserProvider.SCHEME_NAME + ":///";
+    conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, ourUrl);
+    checkSpecificProvider(conf, ourUrl);
+    // see if the credentials are actually in the UGI
+    Credentials credentials =
+        UserGroupInformation.getCurrentUser().getCredentials();
+    assertArrayEquals(new byte[]{1},
+        credentials.getSecretKey(new Text("key4@0")));
+    assertArrayEquals(new byte[]{2},
+        credentials.getSecretKey(new Text("key4@1")));
+  }
+
+  @Test
+  public void testJksProvider() throws Exception {
+    Configuration conf = new Configuration();
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks";
+    File file = new File(tmpDir, "test.jks");
+    file.delete();
+    conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, ourUrl);
+    checkSpecificProvider(conf, ourUrl);
+    assertTrue(file + " should exist", file.isFile());
+  }
+}

+ 16 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java

@@ -258,6 +258,22 @@ public class TestHarFileSystemBasics {
                  0, expectedFileNames.size());
   }
 
+  @Test
+  public void testMakeQualifiedPath() throws Exception {
+    // Construct a valid har file system path with authority that
+    // contains userinfo and port. The userinfo and port are useless
+    // in local fs uri. They are only used to verify har file system
+    // can correctly preserve the information for the underlying file system.
+    String harPathWithUserinfo = "har://file-user:passwd@localhost:80"
+        + harPath.toUri().getPath().toString();
+    Path path = new Path(harPathWithUserinfo);
+    Path qualifiedPath = path.getFileSystem(conf).makeQualified(path);
+    assertTrue(String.format(
+        "The qualified path (%s) did not match the expected path (%s).",
+        qualifiedPath.toString(), harPathWithUserinfo),
+        qualifiedPath.toString().equals(harPathWithUserinfo));
+  }
+
   // ========== Negative:
 
   @Test

+ 13 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

@@ -66,6 +66,8 @@ import org.mockito.internal.util.reflection.Whitebox;
 import org.mortbay.jetty.Connector;
 import org.mortbay.util.ajax.JSON;
 
+import static org.mockito.Mockito.*;
+
 public class TestHttpServer extends HttpServerFunctionalTest {
   static final Log LOG = LogFactory.getLog(TestHttpServer.class);
   private static HttpServer server;
@@ -588,4 +590,15 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     assertEquals(conn.getHeaderField("Expires"), conn.getHeaderField("Date"));
   }
 
+  /**
+   * HTTPServer.Builder should proceed if a external connector is available.
+   */
+  @Test
+  public void testHttpServerBuilderWithExternalConnector() throws Exception {
+    Connector c = mock(Connector.class);
+    doReturn("localhost").when(c).getHost();
+    HttpServer s = new HttpServer.Builder().setName("test").setConnector(c)
+        .build();
+    s.stop();
+  }
 }

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java

@@ -957,6 +957,7 @@ public class TestRPC {
       proxy.sleep(pingInterval*4);
     } finally {
       if (proxy != null) RPC.stopProxy(proxy);
+      server.stop();
     }
   }
 

+ 3 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java

@@ -137,7 +137,9 @@ public class TestSaslRPC {
     LOG.info("Testing QOP:"+expectedQop);
     LOG.info("---------------------------------");
     conf = new Configuration();
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, KERBEROS.toString());
+    // the specific tests for kerberos will enable kerberos.  forcing it
+    // for all tests will cause tests to fail if the user has a TGT
+    conf.set(HADOOP_SECURITY_AUTHENTICATION, SIMPLE.toString());
     conf.set("hadoop.rpc.protection", expectedQop.name().toLowerCase());
     UserGroupInformation.setConfiguration(conf);
     enableSecretManager = null;

+ 87 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSourceAdapter.java

@@ -0,0 +1,87 @@
+/**
+ * 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.metrics2.impl;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsTag;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.MetricsAnnotations;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MetricsSourceBuilder;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.junit.Test;
+
+public class TestMetricsSourceAdapter {
+
+  @Test
+  public void testGetMetricsAndJmx() throws Exception {
+    // create test source with a single metric counter of value 0
+    TestSource source = new TestSource("test");
+    MetricsSourceBuilder sb = MetricsAnnotations.newSourceBuilder(source);
+    final MetricsSource s = sb.build();
+
+    List<MetricsTag> injectedTags = new ArrayList<MetricsTag>();
+    MetricsSourceAdapter sa = new MetricsSourceAdapter(
+        "test", "test", "test desc", s, injectedTags, null, null, 1, false);
+
+    // all metrics are initially assumed to have changed
+    MetricsCollectorImpl builder = new MetricsCollectorImpl();
+    Iterable<MetricsRecordImpl> metricsRecords = sa.getMetrics(builder, true);
+
+    // Validate getMetrics and JMX initial values
+    MetricsRecordImpl metricsRecord = metricsRecords.iterator().next();
+    assertEquals(0L,
+        metricsRecord.metrics().iterator().next().value().longValue());
+
+    Thread.sleep(100); // skip JMX cache TTL
+    assertEquals(0L, (Number)sa.getAttribute("C1"));
+
+    // change metric value
+    source.incrementCnt();
+
+    // validate getMetrics and JMX
+    builder = new MetricsCollectorImpl();
+    metricsRecords = sa.getMetrics(builder, true);
+    metricsRecord = metricsRecords.iterator().next();
+    assertTrue(metricsRecord.metrics().iterator().hasNext());
+    Thread.sleep(100); // skip JMX cache TTL
+    assertEquals(1L, (Number)sa.getAttribute("C1"));
+  }
+
+  @SuppressWarnings("unused")
+  @Metrics(context="test")
+  private static class TestSource {
+    @Metric("C1 desc") MutableCounterLong c1;
+    final MetricsRegistry registry;
+
+    TestSource(String recName) {
+      registry = new MetricsRegistry(recName);
+    }
+
+    public void incrementCnt() {
+      c1.incr();
+    }
+  }
+}

+ 32 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java

@@ -420,7 +420,8 @@ public class TestDomainSocket {
    * @throws IOException
    */
   void testClientServer1(final Class<? extends WriteStrategy> writeStrategyClass,
-      final Class<? extends ReadStrategy> readStrategyClass) throws Exception {
+      final Class<? extends ReadStrategy> readStrategyClass,
+      final DomainSocket preConnectedSockets[]) throws Exception {
     final String TEST_PATH = new File(sockDir.getDir(),
         "test_sock_client_server1").getAbsolutePath();
     final byte clientMsg1[] = new byte[] { 0x1, 0x2, 0x3, 0x4, 0x5, 0x6 };
@@ -428,13 +429,15 @@ public class TestDomainSocket {
     final byte clientMsg2 = 0x45;
     final ArrayBlockingQueue<Throwable> threadResults =
         new ArrayBlockingQueue<Throwable>(2);
-    final DomainSocket serv = DomainSocket.bindAndListen(TEST_PATH);
+    final DomainSocket serv = (preConnectedSockets != null) ?
+      null : DomainSocket.bindAndListen(TEST_PATH);
     Thread serverThread = new Thread() {
       public void run(){
         // Run server
         DomainSocket conn = null;
         try {
-          conn = serv.accept();
+          conn = preConnectedSockets != null ?
+                    preConnectedSockets[0] : serv.accept();
           byte in1[] = new byte[clientMsg1.length];
           ReadStrategy reader = readStrategyClass.newInstance();
           reader.init(conn);
@@ -459,7 +462,8 @@ public class TestDomainSocket {
     Thread clientThread = new Thread() {
       public void run(){
         try {
-          DomainSocket client = DomainSocket.connect(TEST_PATH);
+          DomainSocket client = preConnectedSockets != null ?
+                preConnectedSockets[1] : DomainSocket.connect(TEST_PATH);
           WriteStrategy writer = writeStrategyClass.newInstance();
           writer.init(client);
           writer.write(clientMsg1);
@@ -487,25 +491,45 @@ public class TestDomainSocket {
     }
     serverThread.join(120000);
     clientThread.join(120000);
-    serv.close();
+    if (serv != null) {
+      serv.close();
+    }
   }
 
   @Test(timeout=180000)
   public void testClientServerOutStreamInStream() throws Exception {
     testClientServer1(OutputStreamWriteStrategy.class,
-        InputStreamReadStrategy.class);
+        InputStreamReadStrategy.class, null);
+  }
+
+  @Test(timeout=180000)
+  public void testClientServerOutStreamInStreamWithSocketpair() throws Exception {
+    testClientServer1(OutputStreamWriteStrategy.class,
+        InputStreamReadStrategy.class, DomainSocket.socketpair());
   }
 
   @Test(timeout=180000)
   public void testClientServerOutStreamInDbb() throws Exception {
     testClientServer1(OutputStreamWriteStrategy.class,
-        DirectByteBufferReadStrategy.class);
+        DirectByteBufferReadStrategy.class, null);
+  }
+
+  @Test(timeout=180000)
+  public void testClientServerOutStreamInDbbWithSocketpair() throws Exception {
+    testClientServer1(OutputStreamWriteStrategy.class,
+        DirectByteBufferReadStrategy.class, DomainSocket.socketpair());
   }
 
   @Test(timeout=180000)
   public void testClientServerOutStreamInAbb() throws Exception {
     testClientServer1(OutputStreamWriteStrategy.class,
-        ArrayBackedByteBufferReadStrategy.class);
+        ArrayBackedByteBufferReadStrategy.class, null);
+  }
+
+  @Test(timeout=180000)
+  public void testClientServerOutStreamInAbbWithSocketpair() throws Exception {
+    testClientServer1(OutputStreamWriteStrategy.class,
+        ArrayBackedByteBufferReadStrategy.class, DomainSocket.socketpair());
   }
 
   static private class PassedFile {

+ 19 - 4
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java

@@ -85,6 +85,7 @@ class OpenFileCtx {
   private volatile boolean activeState;
   // The stream write-back status. True means one thread is doing write back.
   private volatile boolean asyncStatus;
+  private volatile long asyncWriteBackStartOffset;
 
   /**
    * The current offset of the file in HDFS. All the content before this offset
@@ -209,6 +210,7 @@ class OpenFileCtx {
     updateLastAccessTime();
     activeState = true;
     asyncStatus = false;
+    asyncWriteBackStartOffset = 0;
     dumpOut = null;
     raf = null;
     nonSequentialWriteInMemory = new AtomicLong(0);
@@ -580,6 +582,7 @@ class OpenFileCtx {
               + nextOffset.get());
         }
         asyncStatus = true;
+        asyncWriteBackStartOffset = writeCtx.getOffset();
         asyncDataService.execute(new AsyncDataService.WriteBackTask(this));
       } else {
         if (LOG.isDebugEnabled()) {
@@ -903,9 +906,11 @@ class OpenFileCtx {
   /** Invoked by AsynDataService to write back to HDFS */
   void executeWriteBack() {
     Preconditions.checkState(asyncStatus,
-        "The openFileCtx has false async status");
+        "openFileCtx has false asyncStatus, fileId:" + latestAttr.getFileid());
+    final long startOffset = asyncWriteBackStartOffset;  
     try {
       while (activeState) {
+        // asyncStatus could be changed to false in offerNextToWrite()
         WriteCtx toWrite = offerNextToWrite();
         if (toWrite != null) {
           // Do the write
@@ -921,8 +926,18 @@ class OpenFileCtx {
             + latestAttr.getFileId());
       }
     } finally {
-      // make sure we reset asyncStatus to false
-      asyncStatus = false;
+      // Make sure to reset asyncStatus to false unless a race happens
+      synchronized (this) {
+        if (startOffset == asyncWriteBackStartOffset) {
+          asyncStatus = false;
+        } else {
+          LOG.info("Another asyn task is already started before this one"
+              + " is finalized. fileId:" + latestAttr.getFileid()
+              + " asyncStatus:" + asyncStatus + " original startOffset:"
+              + startOffset + " new startOffset:" + asyncWriteBackStartOffset
+              + ". Won't change asyncStatus here.");
+        }
+      }
     }
   }
 
@@ -1177,4 +1192,4 @@ class OpenFileCtx {
     return String.format("activeState: %b asyncStatus: %b nextOffset: %d",
         activeState, asyncStatus, nextOffset.get());
   }
-}
+}

+ 246 - 20
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -13,6 +13,10 @@ Trunk (Unreleased)
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
 
+    HDFS-2832. Heterogeneous Storages support in HDFS phase 1 - treat DataNode
+    as a collection of storages (see breakdown of tasks below for features and
+    contributors).
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
@@ -212,43 +216,48 @@ Trunk (Unreleased)
     and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
     (jing9 via szetszwo)
 
-    HDFS-5538. URLConnectionFactory should pick up the SSL related configuration 
-    by default. (Haohui Mai via jing9)
-
     HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
     with DirectoryWithQuotaFeature.  (szetszwo)
 
     HDFS-5556. Add some more NameNode cache statistics, cache pool stats
     (cmccabe)
 
-    HDFS-5545. Allow specifying endpoints for listeners in HttpServer. (Haohui
-    Mai via jing9)
-
     HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
 
     HDFS-5430. Support TTL on CacheDirectives. (wang)
 
-    HDFS-5536. Implement HTTP policy for Namenode and DataNode. (Haohui Mai via
-    jing9)
-
     HDFS-5630. Hook up cache directive and pool usage statistics. (wang)
 
-    HDFS-5312. Generate HTTP / HTTPS URL in DFSUtil#getInfoServer() based on the 
-    configured http policy. (Haohui Mai via jing9)
-
     HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
     FileWithSnapshotFeature.  (jing9 via szetszwo)
 
-    HDFS-5629. Support HTTPS in JournalNode and SecondaryNameNode. 
-    (Haohui Mai via jing9)
+    HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
+    via jing9)
+
+    HDFS-5632. Flatten INodeDirectory hierarchy: Replace
+    INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
+    (jing9 via szetszwo)
+
+    HDFS-5431. Support cachepool-based limit management in path-based caching
+    (awang via cmccabe)
+
+    HDFS-5636. Enforce a max TTL per cache pool. (awang via cmccabe)
+
+    HDFS-5651. Remove dfs.namenode.caching.enabled and improve CRM locking.
+    (cmccabe via wang)
 
     HDFS-5496. Make replication queue initialization asynchronous. (Vinay via
     jing9)
 
   OPTIMIZATIONS
+
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
+    HDFS-5665. Remove the unnecessary writeLock while initializing CacheManager
+    in FsNameSystem Ctor. (Uma Maheswara Rao G via Andrew Wang)
+
   BUG FIXES
+
     HADOOP-9635 Fix potential Stack Overflow in DomainSocket.c (V. Karthik Kumar
                 via cmccabe)
 
@@ -443,6 +452,150 @@ Trunk (Unreleased)
 
     HDFS-5626. dfsadmin -report shows incorrect cache values. (cmccabe)
 
+    HDFS-5406. Send incremental block reports for all storages in a
+    single call. (Arpit Agarwal)
+
+    HDFS-5454. DataNode UUID should be assigned prior to FsDataset
+    initialization. (Arpit Agarwal)
+
+    HDFS-5679. TestCacheDirectives should handle the case where native code
+    is not available. (wang)
+
+    HDFS-5701. Fix the CacheAdmin -addPool -maxTtl option name.
+    (Stephen Chu via wang)
+
+    HDFS-5708. The CacheManager throws a NPE in the DataNode logs when
+    processing cache reports that refer to a block not known to the
+    BlockManager. (cmccabe via wang)
+
+    HDFS-5659. dfsadmin -report doesn't output cache information properly.
+    (wang)
+
+  BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
+
+    HDFS-4985. Add storage type to the protocol and expose it in block report
+    and block locations. (Arpit Agarwal)
+
+    HDFS-5115. Make StorageID a UUID. (Arpit Agarwal)
+
+    HDFS-5000. DataNode configuration should allow specifying storage type.
+    (Arpit Agarwal)
+
+    HDFS-4987. Namenode changes to track multiple storages per datanode.
+    (szetszwo)
+
+    HDFS-5154. Fix TestBlockManager and TestDatanodeDescriptor after HDFS-4987.
+    (Junping Du via szetszwo)
+
+    HDFS-5009. Include storage information in the LocatedBlock.  (szetszwo)
+
+    HDFS-5134. Move blockContentsStale, heartbeatedSinceFailover and
+    firstBlockReport from DatanodeDescriptor to DatanodeStorageInfo; and
+    fix a synchronization problem in DatanodeStorageInfo.  (szetszwo)
+
+    HDFS-5157. Add StorageType to FsVolume.  (Junping Du via szetszwo)
+
+    HDFS-4990. Change BlockPlacementPolicy to choose storages instead of
+    datanodes.  (szetszwo)
+
+    HDFS-5232. Protocol changes to transmit StorageUuid. (Arpit Agarwal)
+
+    HDFS-5233. Use Datanode UUID to identify Datanodes. (Arpit Agarwal)
+
+    HDFS-5222. Move block schedule information from DatanodeDescriptor to
+    DatanodeStorageInfo.  (szetszwo)
+
+    HDFS-4988. Datanode must support all the volumes as individual storages.
+    (Arpit Agarwal)
+
+    HDFS-5377. Heartbeats from Datandode should include one storage report
+    per storage directory. (Arpit Agarwal)
+
+    HDFS-5398. NameNode changes to process storage reports per storage
+    directory. (Arpit Agarwal)
+
+    HDFS-5390. Send one incremental block report per storage directory.
+    (Arpit Agarwal)
+
+    HDFS-5401. Fix NPE in Directory Scanner. (Arpit Agarwal)
+
+    HDFS-5417. Fix storage IDs in PBHelper and UpgradeUtilities.  (szetszwo)
+
+    HDFS-5214. Fix NPEs in BlockManager and DirectoryScanner. (Arpit Agarwal)
+
+    HDFS-5435. File append fails to initialize storageIDs. (Junping Du via
+    Arpit Agarwal)
+
+    HDFS-5437. Fix TestBlockReport and TestBPOfferService failures. (Arpit
+    Agarwal)
+
+    HDFS-5447. Fix TestJspHelper. (Arpit Agarwal)
+
+    HDFS-5452. Fix TestReplicationPolicy and TestBlocksScheduledCounter.
+
+    HDFS-5448. Datanode should generate its ID on first registration. (Arpit
+    Agarwal)
+
+    HDFS-5448. Fix break caused by previous checkin for HDFS-5448. (Arpit
+    Agarwal)
+
+    HDFS-5455. NN should update storageMap on first heartbeat. (Arpit Agarwal)
+
+    HDFS-5457. Fix TestDatanodeRegistration, TestFsck and TestAddBlockRetry.
+    (Contributed by szetszwo)
+
+    HDFS-5466. Update storage IDs when the pipeline is updated. (Contributed
+    by szetszwo)
+
+    HDFS-5439. Fix TestPendingReplication. (Contributed by Junping Du, Arpit
+    Agarwal)
+
+    HDFS-5470. Add back trunk's reportDiff algorithm to the branch.
+    (Contributed by szetszwo)
+
+    HDFS-5472. Fix TestDatanodeManager, TestSafeMode and
+    TestNNThroughputBenchmark (Contributed by szetszwo)
+
+    HDFS-5475. NN incorrectly tracks more than one replica per DN. (Arpit
+    Agarwal)
+
+    HDFS-5481. Fix TestDataNodeVolumeFailure in branch HDFS-2832. (Contributed
+    by Junping Du)
+
+    HDFS-5480. Update Balancer for HDFS-2832. (Contributed by szetszwo)
+
+    HDFS-5486. Fix TestNameNodeMetrics for HDFS-2832. (Arpit Agarwal)
+
+    HDFS-5491. Update editsStored for HDFS-2832. (Arpit Agarwal)
+
+    HDFS-5494. Fix findbugs warnings for HDFS-2832. (Arpit Agarwal)
+
+    HDFS-5508. Fix compilation error after merge. (Contributed by szetszwo)
+
+    HDFS-5501. Fix pendingReceivedRequests tracking in BPServiceActor. (Arpit
+    Agarwal)
+
+    HDFS-5510. Fix a findbug warning in DataStorage.java on HDFS-2832 branch.
+    (Junping Du via Arpit Agarwal)
+ 
+    HDFS-5515. Fix TestDFSStartupVersions for HDFS-2832. (Arpit Agarwal)
+
+    HDFS-5527. Fix TestUnderReplicatedBlocks on branch HDFS-2832. (Arpit
+    Agarwal)
+
+    HDFS-5547. Fix build break after merge from trunk to HDFS-2832. (Arpit
+    Agarwal)
+
+    HDFS-5542. Fix TODO and clean up the code in HDFS-2832. (Contributed by
+    szetszwo)
+
+    HDFS-5559. Fix TestDatanodeConfig in HDFS-2832. (Contributed by szetszwo)
+
+    HDFS-5484. StorageType and State in DatanodeStorageInfo in NameNode is
+    not accurate. (Eric Sirianni via Arpit Agarwal)
+
+    HDFS-5648. Get rid of FsDatasetImpl#perVolumeReplicaMap. (Arpit Agarwal)
+
 Release 2.4.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -483,9 +636,6 @@ Release 2.4.0 - UNRELEASED
     HDFS-5004. Add additional JMX bean for NameNode status data
     (Trevor Lorimer via cos)
 
-    HDFS-5068. Convert NNThroughputBenchmark to a Tool to allow generic options.
-    (shv)
-
     HDFS-4994. Audit log getContentSummary() calls. (Robert Parker via kihwal)
 
     HDFS-5144. Document time unit to NameNodeMetrics. (Akira Ajisaka via
@@ -590,6 +740,49 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5633. Improve OfflineImageViewer to use less memory. (jing9)
 
+    HDFS-5023. TestSnapshotPathINodes.testAllowSnapshot is failing with jdk7
+    (Mit Desai via jeagles)
+
+    HDFS-5637. Try to refeatchToken while local read InvalidToken occurred.
+    (Liang Xie via junping_du)
+
+    HDFS-5652. Refactor invalid block token exception handling in DFSInputStream.
+    (Liang Xie via junping_du)
+
+    HDFS-5350. Name Node should report fsimage transfer time as a metric.
+    (Jimmy Xiang via wang)
+
+    HDFS-5538. URLConnectionFactory should pick up the SSL related configuration 
+    by default. (Haohui Mai via jing9)
+
+    HDFS-5545. Allow specifying endpoints for listeners in HttpServer. (Haohui
+    Mai via jing9)
+
+    HDFS-5536. Implement HTTP policy for Namenode and DataNode. (Haohui Mai via
+    jing9)
+
+    HDFS-5312. Generate HTTP / HTTPS URL in DFSUtil#getInfoServer() based on the 
+    configured http policy. (Haohui Mai via jing9)
+
+    HDFS-5629. Support HTTPS in JournalNode and SecondaryNameNode. 
+    (Haohui Mai via jing9)    
+
+    HDFS-5674. Editlog code cleanup: remove @SuppressWarnings("deprecation") in
+    FSEditLogOp; change FSEditLogOpCodes.fromByte(..) to be more efficient; and
+    change Some fields in FSEditLog to final.  (szetszwo)
+
+    HDFS-5634. Allow BlockReaderLocal to switch between checksumming and not
+    (cmccabe)
+
+    HDFS-5663 make the retry time and interval value configurable in openInfo()
+    (Liang Xie via stack)
+
+    HDFS-5540. Fix intermittent failure in TestBlocksWithNotEnoughRacks.
+    (Binglin Chang via junping_du)
+
+    HDFS-2933. Improve DataNode Web UI Index Page. (Vivek Ganesan via
+    Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -597,6 +790,8 @@ Release 2.4.0 - UNRELEASED
     HDFS-5341. Reduce fsdataset lock duration during directory scanning.
     (Qus-Jiawei via kihwal)
 
+    HDFS-5681. renewLease should not hold fsn write lock. (daryn via Kihwal)
+
   BUG FIXES
 
     HDFS-5034.  Remove debug prints from GetFileLinkInfo (Andrew Wang via Colin
@@ -645,6 +840,14 @@ Release 2.4.0 - UNRELEASED
     HDFS-5580. Fix infinite loop in Balancer.waitForMoveCompletion.
     (Binglin Chang via junping_du)
 
+    HDFS-5676. fix inconsistent synchronization of CachingStrategy (cmccabe)
+
+    HDFS-5691. Fix typo in ShortCircuitLocalRead document.
+    (Akira Ajisaka via suresh)
+
+    HDFS-5690. DataNode fails to start in secure mode when dfs.http.policy equals to 
+    HTTP_ONLY. (Haohui Mai via jing9)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -672,6 +875,18 @@ Release 2.3.0 - UNRELEASED
     HDFS-4983. Numeric usernames do not work with WebHDFS FS. (Yongjun Zhang via
     jing9)
 
+    HDFS-5592. statechangeLog of completeFile should be logged only in case of success. 
+    (Vinayakumar via umamahesh)
+
+    HDFS-5662. Can't decommission a DataNode due to file's replication factor
+    larger than the rest of the cluster size. (brandonli)
+
+    HDFS-5068. Convert NNThroughputBenchmark to a Tool to allow generic options.
+    (shv)
+
+    HDFS-5675. Add Mkdirs operation to NNThroughputBenchmark.
+    (Plamen Jeliazkov via shv)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -813,6 +1028,20 @@ Release 2.3.0 - UNRELEASED
     HDFS-5074. Allow starting up from an fsimage checkpoint in the middle of a
     segment. (Todd Lipcon via atm)
 
+    HDFS-4201. NPE in BPServiceActor#sendHeartBeat. (jxiang via cmccabe)
+
+    HDFS-5666. Fix inconsistent synchronization in BPOfferService (jxiang via cmccabe)
+    
+    HDFS-5657. race condition causes writeback state error in NFS gateway (brandonli)
+
+    HDFS-5661. Browsing FileSystem via web ui, should use datanode's fqdn instead of ip 
+    address. (Benoy Antony via jing9)
+
+    HDFS-5582. hdfs getconf -excludeFile or -includeFile always failed (sathish
+    via cmccabe)
+
+    HDFS-5671. Fix socket leak in DFSInputStream#getBlockReader. (JamesLi via umamahesh) 
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES
@@ -942,9 +1171,6 @@ Release 2.1.1-beta - 2013-09-23
     HDFS-5047. Supress logging of full stack trace of quota and lease
     exceptions. (Robert Parker via kihwal)
 
-    HDFS-2933. Improve DataNode Web UI Index Page. (Vivek Ganesan via
-    Arpit Agarwal)
-
     HDFS-5111. Remove duplicated error message for snapshot commands when 
     processing invalid arguments. (jing9)
 

+ 1 - 8
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -357,16 +357,9 @@
       <Method name="insertInternal" />
       <Bug pattern="BC_UNCONFIRMED_CAST" />
     </Match>
-    <!-- These two are used for shutting down and kicking the CRMon, do not need strong sync -->
     <Match>
       <Class name="org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor" />
-      <Field name="shutdown" />
-      <Bug pattern="IS2_INCONSISTENT_SYNC" />
-    </Match>
-    <Match>
-      <Class name="org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor" />
-      <Field name="rescanImmediately" />
-      <Bug pattern="IS2_INCONSISTENT_SYNC" />
+      <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
     </Match>
 
  </FindBugsFilter>

+ 17 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/RMNotYetActiveException.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/CacheFlag.java

@@ -15,22 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.hadoop.yarn.exceptions;
+package org.apache.hadoop.fs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * Exception to be thrown when an Active-Only operation is attempted on a
- * ResourceManager that is not Active.
+ * Specifies semantics for CacheDirective operations. Multiple flags can
+ * be combined in an EnumSet.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class RMNotYetActiveException extends YarnException {
-  private static final long serialVersionUID = 1L;
+public enum CacheFlag {
+
+  /**
+   * Ignore cache pool resource limits when performing this operation.
+   */
+  FORCE((short) 0x01);
+  private final short mode;
+
+  private CacheFlag(short mode) {
+    this.mode = mode;
+  }
 
-  public RMNotYetActiveException() {
-    super("ResourceManager is not yet Active!");
+  short getMode() {
+    return mode;
   }
 }

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java

@@ -18,8 +18,10 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
+import java.util.EnumSet;
 
 import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.client.ClientMmap;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -89,10 +91,10 @@ public interface BlockReader extends ByteBufferReadable {
   /**
    * Get a ClientMmap object for this BlockReader.
    *
-   * @param curBlock      The current block.
+   * @param opts          The read options to use.
    * @return              The ClientMmap object, or null if mmap is not
    *                      supported.
    */
-  ClientMmap getClientMmap(LocatedBlock curBlock,
+  ClientMmap getClientMmap(EnumSet<ReadOption> opts,
         ClientMmapManager mmapManager);
 }

+ 18 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
@@ -98,7 +99,7 @@ public class BlockReaderFactory {
         // enabled, try to set up a BlockReaderLocal.
         BlockReader reader = newShortCircuitBlockReader(conf, file,
             block, blockToken, startOffset, len, peer, datanodeID,
-            domSockFactory, verifyChecksum, fisCache);
+            domSockFactory, verifyChecksum, fisCache, cachingStrategy);
         if (reader != null) {
           // One we've constructed the short-circuit block reader, we don't
           // need the socket any more.  So let's return it to the cache.
@@ -160,7 +161,8 @@ public class BlockReaderFactory {
    * @param verifyChecksum     True if we should verify the checksums.
    *                           Note: even if this is true, when
    *                           DFS_CLIENT_READ_CHECKSUM_SKIP_CHECKSUM_KEY is
-   *                           set, we will skip checksums.
+   *                           set or the block is mlocked, we will skip
+   *                           checksums.
    *
    * @return                   The BlockReaderLocal, or null if the
    *                           DataNode declined to provide short-circuit
@@ -172,7 +174,8 @@ public class BlockReaderFactory {
       Token<BlockTokenIdentifier> blockToken, long startOffset,
       long len, Peer peer, DatanodeID datanodeID,
       DomainSocketFactory domSockFactory, boolean verifyChecksum,
-      FileInputStreamCache fisCache) throws IOException {
+      FileInputStreamCache fisCache,
+      CachingStrategy cachingStrategy) throws IOException {
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(
           peer.getOutputStream()));
@@ -189,9 +192,18 @@ public class BlockReaderFactory {
       FileInputStream fis[] = new FileInputStream[2];
       sock.recvFileInputStreams(fis, buf, 0, buf.length);
       try {
-        reader = new BlockReaderLocal(conf, file, block,
-            startOffset, len, fis[0], fis[1], datanodeID, verifyChecksum,
-            fisCache);
+        reader = new BlockReaderLocal.Builder(conf).
+            setFilename(file).
+            setBlock(block).
+            setStartOffset(startOffset).
+            setStreams(fis).
+            setDatanodeID(datanodeID).
+            setVerifyChecksum(verifyChecksum).
+            setBlockMetadataHeader(
+                BlockMetadataHeader.preadHeader(fis[1].getChannel())).
+            setFileInputStreamCache(fisCache).
+            setCachingStrategy(cachingStrategy).
+            build();
       } finally {
         if (reader == null) {
           IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);

文件差異過大導致無法顯示
+ 530 - 393
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java


+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java

@@ -24,10 +24,12 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
+import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.client.ClientMmap;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.commons.logging.Log;
@@ -706,8 +708,8 @@ class BlockReaderLocalLegacy implements BlockReader {
   }
 
   @Override
-  public ClientMmap getClientMmap(LocatedBlock curBlock,
-      ClientMmapManager mmapManager) {
+  public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
+        ClientMmapManager mmapManager) {
     return null;
   }
 }

+ 15 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -85,6 +85,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStorageLocation;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -98,10 +99,10 @@ import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -109,6 +110,7 @@ import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
@@ -121,7 +123,6 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -282,6 +283,8 @@ public class DFSClient implements java.io.Closeable {
     final boolean getHdfsBlocksMetadataEnabled;
     final int getFileBlockStorageLocationsNumThreads;
     final int getFileBlockStorageLocationsTimeout;
+    final int retryTimesForGetLastBlockLength;
+    final int retryIntervalForGetLastBlockLength;
 
     final boolean useLegacyBlockReader;
     final boolean useLegacyBlockReaderLocal;
@@ -355,6 +358,12 @@ public class DFSClient implements java.io.Closeable {
       getFileBlockStorageLocationsTimeout = conf.getInt(
           DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT,
           DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_DEFAULT);
+      retryTimesForGetLastBlockLength = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH,
+          DFSConfigKeys.DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
+      retryIntervalForGetLastBlockLength = conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_RETRY_INTERVAL_GET_LAST_BLOCK_LENGTH,
+        DFSConfigKeys.DFS_CLIENT_RETRY_INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT);
 
       useLegacyBlockReader = conf.getBoolean(
           DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER,
@@ -2295,20 +2304,20 @@ public class DFSClient implements java.io.Closeable {
   }
 
   public long addCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
     try {
-      return namenode.addCacheDirective(info);
+      return namenode.addCacheDirective(info, flags);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }
   }
   
   public void modifyCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
     try {
-      namenode.modifyCacheDirective(info);
+      namenode.modifyCacheDirective(info, flags);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }

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

@@ -65,6 +65,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS_DEFAULT = 10;
   public static final String  DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT = "dfs.client.file-block-storage-locations.timeout";
   public static final int     DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_DEFAULT = 60;
+  public static final String  DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH = "dfs.client.retry.times.get-last-block-length";
+  public static final int     DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT = 3;
+  public static final String  DFS_CLIENT_RETRY_INTERVAL_GET_LAST_BLOCK_LENGTH = "dfs.client.retry.interval-ms.get-last-block-length";
+  public static final int     DFS_CLIENT_RETRY_INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT = 4000;
 
   // HA related configuration
   public static final String  DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX = "dfs.client.failover.proxy.provider";
@@ -104,8 +108,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long    DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT = 0;
   public static final String  DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY = "dfs.datanode.fsdatasetcache.max.threads.per.volume";
   public static final int     DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT = 4;
-  public static final String  DFS_NAMENODE_CACHING_ENABLED_KEY = "dfs.namenode.caching.enabled";
-  public static final boolean DFS_NAMENODE_CACHING_ENABLED_DEFAULT = false;
+  public static final String  DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT =
+    "dfs.namenode.path.based.cache.block.map.allocation.percent";
+  public static final float    DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT_DEFAULT = 0.25f;
 
   public static final String  DFS_NAMENODE_HTTP_PORT_KEY = "dfs.http.port";
   public static final int     DFS_NAMENODE_HTTP_PORT_DEFAULT = 50070;

+ 89 - 51
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.io.ByteBufferPool;
@@ -65,6 +66,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.IdentityHashStore;
 
@@ -226,7 +228,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         dfsClient.getConf().shortCircuitStreamsCacheSize,
         dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
     this.cachingStrategy =
-        dfsClient.getDefaultReadCachingStrategy().duplicate();
+        dfsClient.getDefaultReadCachingStrategy();
     openInfo();
   }
 
@@ -235,7 +237,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    */
   synchronized void openInfo() throws IOException, UnresolvedLinkException {
     lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
-    int retriesForLastBlockLength = 3;
+    int retriesForLastBlockLength = dfsClient.getConf().retryTimesForGetLastBlockLength;
     while (retriesForLastBlockLength > 0) {
       // Getting last block length as -1 is a special case. When cluster
       // restarts, DNs may not report immediately. At this time partial block
@@ -245,7 +247,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         DFSClient.LOG.warn("Last block locations not available. "
             + "Datanodes might not have reported blocks completely."
             + " Will retry for " + retriesForLastBlockLength + " times");
-        waitFor(4000);
+        waitFor(dfsClient.getConf().retryIntervalForGetLastBlockLength);
         lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
       } else {
         break;
@@ -572,7 +574,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
         blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
             accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
-            buffersize, verifyChecksum, dfsClient.clientName);
+            buffersize, verifyChecksum, dfsClient.clientName, cachingStrategy);
         if(connectFailedOnce) {
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
                              " for " + blk);
@@ -590,20 +592,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // The encryption key used is invalid.
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
-        } else if (ex instanceof InvalidBlockTokenException && refetchToken > 0) {
-          DFSClient.LOG.info("Will fetch a new access token and retry, " 
-              + "access token was invalid when connecting to " + targetAddr
-              + " : " + ex);
-          /*
-           * Get a new access token and retry. Retry is needed in 2 cases. 1)
-           * When both NN and DN re-started while DFSClient holding a cached
-           * access token. 2) In the case that NN fails to update its
-           * access key at pre-set interval (by a wide margin) and
-           * subsequently restarts. In this case, DN re-registers itself with
-           * NN and receives a new access key, but DN will delete the old
-           * access key from its memory since it's considered expired based on
-           * the estimated expiration date.
-           */
+        } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
           refetchToken--;
           fetchBlockAt(target);
         } else {
@@ -939,7 +928,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       // cached block locations may have been updated by chooseDataNode()
       // or fetchBlockAt(). Always get the latest list of locations at the 
       // start of the loop.
-      block = getBlockAt(block.getStartOffset(), false);
+      CachingStrategy curCachingStrategy;
+      synchronized (this) {
+        block = getBlockAt(block.getStartOffset(), false);
+        curCachingStrategy = cachingStrategy;
+      }
       DNAddrPair retval = chooseDataNode(block);
       DatanodeInfo chosenNode = retval.info;
       InetSocketAddress targetAddr = retval.addr;
@@ -951,7 +944,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         int len = (int) (end - start + 1);
         reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
             blockToken, start, len, buffersize, verifyChecksum,
-            dfsClient.clientName);
+            dfsClient.clientName, curCachingStrategy);
         int nread = reader.readAll(buf, offset, len);
         if (nread != len) {
           throw new IOException("truncated return from reader.read(): " +
@@ -976,10 +969,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // The encryption key used is invalid.
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
-        } else if (e instanceof InvalidBlockTokenException && refetchToken > 0) {
-          DFSClient.LOG.info("Will get a new access token and retry, "
-              + "access token was invalid when connecting to " + targetAddr
-              + " : " + e);
+        } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
           refetchToken--;
           fetchBlockAt(block.getStartOffset());
           continue;
@@ -1000,6 +990,34 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
   }
 
+  /**
+   * Should the block access token be refetched on an exception
+   * 
+   * @param ex Exception received
+   * @param targetAddr Target datanode address from where exception was received
+   * @return true if block access token has expired or invalid and it should be
+   *         refetched
+   */
+  private static boolean tokenRefetchNeeded(IOException ex,
+      InetSocketAddress targetAddr) {
+    /*
+     * Get a new access token and retry. Retry is needed in 2 cases. 1)
+     * When both NN and DN re-started while DFSClient holding a cached
+     * access token. 2) In the case that NN fails to update its
+     * access key at pre-set interval (by a wide margin) and
+     * subsequently restarts. In this case, DN re-registers itself with
+     * NN and receives a new access key, but DN will delete the old
+     * access key from its memory since it's considered expired based on
+     * the estimated expiration date.
+     */
+    if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
+      DFSClient.LOG.info("Access token was invalid when connecting to "
+          + targetAddr + " : " + ex);
+      return true;
+    }
+    return false;
+  }
+
   private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
     Peer peer = null;
     boolean success = false;
@@ -1039,6 +1057,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * @param bufferSize  The IO buffer size (not the client buffer size)
    * @param verifyChecksum  Whether to verify checksum
    * @param clientName  Client name
+   * @param CachingStrategy  caching strategy to use
    * @return New BlockReader instance
    */
   protected BlockReader getBlockReader(InetSocketAddress dnAddr,
@@ -1050,7 +1069,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
                                        long len,
                                        int bufferSize,
                                        boolean verifyChecksum,
-                                       String clientName)
+                                       String clientName,
+                                       CachingStrategy curCachingStrategy)
       throws IOException {
     // Firstly, we check to see if we have cached any file descriptors for
     // local blocks.  If so, we can just re-use those file descriptors.
@@ -1060,9 +1080,18 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         DFSClient.LOG.debug("got FileInputStreams for " + block + " from " +
             "the FileInputStreamCache.");
       }
-      return new BlockReaderLocal(dfsClient.getConf(), file,
-        block, startOffset, len, fis[0], fis[1], chosenNode, verifyChecksum,
-        fileInputStreamCache);
+      return new BlockReaderLocal.Builder(dfsClient.getConf()).
+          setFilename(file).
+          setBlock(block).
+          setStartOffset(startOffset).
+          setStreams(fis).
+          setDatanodeID(chosenNode).
+          setVerifyChecksum(verifyChecksum).
+          setBlockMetadataHeader(BlockMetadataHeader.
+              preadHeader(fis[1].getChannel())).
+          setFileInputStreamCache(fileInputStreamCache).
+          setCachingStrategy(curCachingStrategy).
+          build();
     }
     
     // If the legacy local block reader is enabled and we are reading a local
@@ -1096,7 +1125,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             len, verifyChecksum, clientName, peer, chosenNode, 
             dsFactory, peerCache, fileInputStreamCache,
-            allowShortCircuitLocalReads, cachingStrategy);
+            allowShortCircuitLocalReads, curCachingStrategy);
         return reader;
       } catch (IOException ex) {
         DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
@@ -1119,7 +1148,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             len, verifyChecksum, clientName, peer, chosenNode,
             dsFactory, peerCache, fileInputStreamCache,
-            allowShortCircuitLocalReads, cachingStrategy);
+            allowShortCircuitLocalReads, curCachingStrategy);
         return reader;
       } catch (IOException e) {
         DFSClient.LOG.warn("failed to connect to " + domSock, e);
@@ -1143,7 +1172,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             len, verifyChecksum, clientName, peer, chosenNode, 
             dsFactory, peerCache, fileInputStreamCache, false,
-            cachingStrategy);
+            curCachingStrategy);
         return reader;
       } catch (IOException ex) {
         DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
@@ -1159,11 +1188,21 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
     // Try to create a new remote peer.
     Peer peer = newTcpPeer(dnAddr);
-    return BlockReaderFactory.newBlockReader(
-        dfsClient.getConf(), file, block, blockToken, startOffset,
-        len, verifyChecksum, clientName, peer, chosenNode, 
-        dsFactory, peerCache, fileInputStreamCache, false,
-        cachingStrategy);
+    try {
+      reader = BlockReaderFactory.newBlockReader(dfsClient.getConf(), file,
+          block, blockToken, startOffset, len, verifyChecksum, clientName,
+          peer, chosenNode, dsFactory, peerCache, fileInputStreamCache, false,
+          curCachingStrategy);
+      return reader;
+    } catch (IOException ex) {
+      DFSClient.LOG.debug(
+          "Exception while getting block reader, closing stale " + peer, ex);
+      throw ex;
+    } finally {
+      if (reader == null) {
+        IOUtils.closeQuietly(peer);
+      }
+    }
   }
 
 
@@ -1344,7 +1383,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
        * deadNodes and added currentNode again. Thats ok. */
       deadNodes.remove(oldNode);
     }
-    if (!oldNode.getStorageID().equals(newNode.getStorageID())) {
+    if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
       currentNode = newNode;
       return true;
     } else {
@@ -1437,14 +1476,18 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   @Override
   public synchronized void setReadahead(Long readahead)
       throws IOException {
-    this.cachingStrategy.setReadahead(readahead);
+    this.cachingStrategy =
+        new CachingStrategy.Builder(this.cachingStrategy).
+            setReadahead(readahead).build();
     closeCurrentBlockReader();
   }
 
   @Override
   public synchronized void setDropBehind(Boolean dropBehind)
       throws IOException {
-    this.cachingStrategy.setDropBehind(dropBehind);
+    this.cachingStrategy =
+        new CachingStrategy.Builder(this.cachingStrategy).
+            setDropBehind(dropBehind).build();
     closeCurrentBlockReader();
   }
 
@@ -1466,23 +1509,19 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             "at position " + pos);
       }
     }
-    boolean canSkipChecksums = opts.contains(ReadOption.SKIP_CHECKSUMS);
-    if (canSkipChecksums) {
-      ByteBuffer buffer = tryReadZeroCopy(maxLength);
-      if (buffer != null) {
-        return buffer;
-      }
+    ByteBuffer buffer = tryReadZeroCopy(maxLength, opts);
+    if (buffer != null) {
+      return buffer;
     }
-    ByteBuffer buffer = ByteBufferUtil.
-        fallbackRead(this, bufferPool, maxLength);
+    buffer = ByteBufferUtil.fallbackRead(this, bufferPool, maxLength);
     if (buffer != null) {
       extendedReadBuffers.put(buffer, bufferPool);
     }
     return buffer;
   }
 
-  private synchronized ByteBuffer tryReadZeroCopy(int maxLength)
-      throws IOException {
+  private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
+      EnumSet<ReadOption> opts) throws IOException {
     // Java ByteBuffers can't be longer than 2 GB, because they use
     // 4-byte signed integers to represent capacity, etc.
     // So we can't mmap the parts of the block higher than the 2 GB offset.
@@ -1505,8 +1544,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     long blockPos = curPos - blockStartInFile;
     long limit = blockPos + length;
     ClientMmap clientMmap =
-        blockReader.getClientMmap(currentLocatedBlock,
-            dfsClient.getMmapManager());
+        blockReader.getClientMmap(opts, dfsClient.getMmapManager());
     if (clientMmap == null) {
       if (DFSClient.LOG.isDebugEnabled()) {
         DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +

+ 39 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -150,7 +150,7 @@ public class DFSOutputStream extends FSOutputSummer
   private Progressable progress;
   private final short blockReplication; // replication factor of file
   private boolean shouldSyncBlock = false; // force blocks to disk upon close
-  private CachingStrategy cachingStrategy;
+  private AtomicReference<CachingStrategy> cachingStrategy;
   private boolean failPacket = false;
   
   private static class Packet {
@@ -312,6 +312,7 @@ public class DFSOutputStream extends FSOutputSummer
     private DataInputStream blockReplyStream;
     private ResponseProcessor response = null;
     private volatile DatanodeInfo[] nodes = null; // list of targets for current block
+    private volatile String[] storageIDs = null;
     private LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes =
         CacheBuilder.newBuilder()
         .expireAfterWrite(
@@ -402,7 +403,7 @@ public class DFSOutputStream extends FSOutputSummer
       }
 
       // setup pipeline to append to the last block XXX retries??
-      nodes = lastBlock.getLocations();
+      setPipeline(lastBlock);
       errorIndex = -1;   // no errors yet.
       if (nodes.length < 1) {
         throw new IOException("Unable to retrieve blocks locations " +
@@ -411,6 +412,14 @@ public class DFSOutputStream extends FSOutputSummer
 
       }
     }
+    
+    private void setPipeline(LocatedBlock lb) {
+      setPipeline(lb.getLocations(), lb.getStorageIDs());
+    }
+    private void setPipeline(DatanodeInfo[] nodes, String[] storageIDs) {
+      this.nodes = nodes;
+      this.storageIDs = storageIDs;
+    }
 
     private void setFavoredNodes(String[] favoredNodes) {
       this.favoredNodes = favoredNodes;
@@ -434,7 +443,7 @@ public class DFSOutputStream extends FSOutputSummer
       this.setName("DataStreamer for file " + src);
       closeResponder();
       closeStream();
-      nodes = null;
+      setPipeline(null, null);
       stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
     }
     
@@ -503,7 +512,7 @@ public class DFSOutputStream extends FSOutputSummer
             if(DFSClient.LOG.isDebugEnabled()) {
               DFSClient.LOG.debug("Allocating new block");
             }
-            nodes = nextBlockOutputStream();
+            setPipeline(nextBlockOutputStream());
             initDataStreaming();
           } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
             if(DFSClient.LOG.isDebugEnabled()) {
@@ -917,9 +926,10 @@ public class DFSOutputStream extends FSOutputSummer
       //get a new datanode
       final DatanodeInfo[] original = nodes;
       final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
-          src, block, nodes, failed.toArray(new DatanodeInfo[failed.size()]),
+          src, block, nodes, storageIDs,
+          failed.toArray(new DatanodeInfo[failed.size()]),
           1, dfsClient.clientName);
-      nodes = lb.getLocations();
+      setPipeline(lb);
 
       //find the new datanode
       final int d = findNewDatanode(original);
@@ -1019,7 +1029,14 @@ public class DFSOutputStream extends FSOutputSummer
           System.arraycopy(nodes, 0, newnodes, 0, errorIndex);
           System.arraycopy(nodes, errorIndex+1, newnodes, errorIndex,
               newnodes.length-errorIndex);
-          nodes = newnodes;
+
+          final String[] newStorageIDs = new String[newnodes.length];
+          System.arraycopy(storageIDs, 0, newStorageIDs, 0, errorIndex);
+          System.arraycopy(storageIDs, errorIndex+1, newStorageIDs, errorIndex,
+              newStorageIDs.length-errorIndex);
+          
+          setPipeline(newnodes, newStorageIDs);
+
           hasError = false;
           lastException.set(null);
           errorIndex = -1;
@@ -1055,7 +1072,8 @@ public class DFSOutputStream extends FSOutputSummer
         // update pipeline at the namenode
         ExtendedBlock newBlock = new ExtendedBlock(
             block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
-        dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock, nodes);
+        dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
+            nodes, storageIDs);
         // update client side generation stamp
         block = newBlock;
       }
@@ -1068,7 +1086,7 @@ public class DFSOutputStream extends FSOutputSummer
      * Must get block ID and the IDs of the destinations from the namenode.
      * Returns the list of target datanodes.
      */
-    private DatanodeInfo[] nextBlockOutputStream() throws IOException {
+    private LocatedBlock nextBlockOutputStream() throws IOException {
       LocatedBlock lb = null;
       DatanodeInfo[] nodes = null;
       int count = dfsClient.getConf().nBlockWriteRetry;
@@ -1110,7 +1128,7 @@ public class DFSOutputStream extends FSOutputSummer
       if (!success) {
         throw new IOException("Unable to create new block.");
       }
-      return nodes;
+      return lb;
     }
 
     // connects to the first datanode in the pipeline
@@ -1165,7 +1183,7 @@ public class DFSOutputStream extends FSOutputSummer
           new Sender(out).writeBlock(block, accessToken, dfsClient.clientName,
               nodes, null, recoveryFlag? stage.getRecoveryStage() : stage, 
               nodes.length, block.getNumBytes(), bytesSent, newGS, checksum,
-              cachingStrategy);
+              cachingStrategy.get());
   
           // receive ack for connect
           BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
@@ -1360,8 +1378,8 @@ public class DFSOutputStream extends FSOutputSummer
     this.blockSize = stat.getBlockSize();
     this.blockReplication = stat.getReplication();
     this.progress = progress;
-    this.cachingStrategy =
-        dfsClient.getDefaultWriteCachingStrategy().duplicate();
+    this.cachingStrategy = new AtomicReference<CachingStrategy>(
+        dfsClient.getDefaultWriteCachingStrategy());
     if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug(
           "Set non-null progress callback on DFSOutputStream " + src);
@@ -1975,7 +1993,14 @@ public class DFSOutputStream extends FSOutputSummer
 
   @Override
   public void setDropBehind(Boolean dropBehind) throws IOException {
-    this.cachingStrategy.setDropBehind(dropBehind);
+    CachingStrategy prevStrategy, nextStrategy;
+    // CachingStrategy is immutable.  So build a new CachingStrategy with the
+    // modifications we want, and compare-and-swap it in.
+    do {
+      prevStrategy = this.cachingStrategy.get();
+      nextStrategy = new CachingStrategy.Builder(prevStrategy).
+                        setDropBehind(dropBehind).build();
+    } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy));
   }
 
   @VisibleForTesting

+ 30 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -145,6 +145,23 @@ public class DFSUtil {
     return SECURE_RANDOM.get();
   }
 
+  /** Shuffle the elements in the given array. */
+  public static <T> T[] shuffle(final T[] array) {
+    if (array != null && array.length > 0) {
+      final Random random = getRandom();
+      for (int n = array.length; n > 1; ) {
+        final int randomIndex = random.nextInt(n);
+        n--;
+        if (n != randomIndex) {
+          final T tmp = array[randomIndex];
+          array[randomIndex] = array[n];
+          array[n] = tmp;
+        }
+      }
+    }
+    return array;
+  }
+
   /**
    * Compartor for sorting DataNodeInfo[] based on decommissioned states.
    * Decommissioned nodes are moved to the end of the array on sorting with
@@ -1529,7 +1546,11 @@ public class DFSUtil {
    * Converts a time duration in milliseconds into DDD:HH:MM:SS format.
    */
   public static String durationToString(long durationMs) {
-    Preconditions.checkArgument(durationMs >= 0, "Invalid negative duration");
+    boolean negative = false;
+    if (durationMs < 0) {
+      negative = true;
+      durationMs = -durationMs;
+    }
     // Chop off the milliseconds
     long durationSec = durationMs / 1000;
     final int secondsPerMinute = 60;
@@ -1542,7 +1563,12 @@ public class DFSUtil {
     final long minutes = durationSec / secondsPerMinute;
     durationSec -= minutes * secondsPerMinute;
     final long seconds = durationSec;
-    return String.format("%03d:%02d:%02d:%02d", days, hours, minutes, seconds);
+    final long milliseconds = durationMs % 1000;
+    String format = "%03d:%02d:%02d:%02d.%03d";
+    if (negative)  {
+      format = "-" + format;
+    }
+    return String.format(format, days, hours, minutes, seconds, milliseconds);
   }
 
   /**
@@ -1554,9 +1580,9 @@ public class DFSUtil {
           + ": too short");
     }
     String ttlString = relTime.substring(0, relTime.length()-1);
-    int ttl;
+    long ttl;
     try {
-      ttl = Integer.parseInt(ttlString);
+      ttl = Long.parseLong(ttlString);
     } catch (NumberFormatException e) {
       throw new IOException("Unable to parse relative time value of " + relTime
           + ": " + ttlString + " is not a number");

+ 25 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStorageLocation;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -1585,40 +1586,56 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
   }
 
+  /**
+   * @see {@link #addCacheDirective(CacheDirectiveInfo, EnumSet)}
+   */
+  public long addCacheDirective(CacheDirectiveInfo info) throws IOException {
+    return addCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
+  }
+
   /**
    * Add a new CacheDirective.
    * 
    * @param info Information about a directive to add.
+   * @param flags {@link CacheFlag}s to use for this operation.
    * @return the ID of the directive that was created.
    * @throws IOException if the directive could not be added
    */
   public long addCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     Preconditions.checkNotNull(info.getPath());
     Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
         makeQualified(getUri(), getWorkingDirectory());
     return dfs.addCacheDirective(
         new CacheDirectiveInfo.Builder(info).
             setPath(path).
-            build());
+            build(),
+        flags);
   }
-  
+
+  /**
+   * @see {@link #modifyCacheDirective(CacheDirectiveInfo, EnumSet)}
+   */
+  public void modifyCacheDirective(CacheDirectiveInfo info) throws IOException {
+    modifyCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
+  }
+
   /**
    * Modify a CacheDirective.
    * 
-   * @param info Information about the directive to modify.
-   *             You must set the ID to indicate which CacheDirective you want
-   *             to modify.
+   * @param info Information about the directive to modify. You must set the ID
+   *          to indicate which CacheDirective you want to modify.
+   * @param flags {@link CacheFlag}s to use for this operation.
    * @throws IOException if the directive could not be modified
    */
   public void modifyCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     if (info.getPath() != null) {
       info = new CacheDirectiveInfo.Builder(info).
           setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
               makeQualified(getUri(), getWorkingDirectory())).build();
     }
-    dfs.modifyCacheDirective(info);
+    dfs.modifyCacheDirective(info, flags);
   }
 
   /**

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

@@ -23,10 +23,12 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FSInputChecker;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.client.ClientMmap;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.net.Peer;
@@ -490,8 +492,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   }
 
   @Override
-  public ClientMmap getClientMmap(LocatedBlock curBlock,
-      ClientMmapManager mmapManager) {
+  public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
+        ClientMmapManager mmapManager) {
     return null;
   }
 }

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

@@ -25,10 +25,12 @@ import java.io.OutputStream;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.ReadableByteChannel;
+import java.util.EnumSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.client.ClientMmap;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.net.Peer;
@@ -455,8 +457,8 @@ public class RemoteBlockReader2  implements BlockReader {
   }
 
   @Override
-  public ClientMmap getClientMmap(LocatedBlock curBlock,
-      ClientMmapManager manager) {
+  public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
+        ClientMmapManager mmapManager) {
     return null;
   }
 }

+ 35 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java

@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Defines the types of supported storage media. The default storage
+ * medium is assumed to be DISK.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum StorageType {
+  DISK,
+  SSD;
+
+  public static StorageType DEFAULT = DISK;
+}

+ 12 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -19,10 +19,12 @@ package org.apache.hadoop.hdfs.client;
 
 import java.io.IOException;
 import java.net.URI;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
@@ -131,25 +133,26 @@ public class HdfsAdmin {
    * Add a new CacheDirectiveInfo.
    * 
    * @param info Information about a directive to add.
+   * @param flags {@link CacheFlag}s to use for this operation.
    * @return the ID of the directive that was created.
    * @throws IOException if the directive could not be added
    */
-  public long addCacheDirective(CacheDirectiveInfo info)
-      throws IOException {
-    return dfs.addCacheDirective(info);
+  public long addCacheDirective(CacheDirectiveInfo info,
+      EnumSet<CacheFlag> flags) throws IOException {
+  return dfs.addCacheDirective(info, flags);
   }
   
   /**
    * Modify a CacheDirective.
    * 
-   * @param info Information about the directive to modify.
-   *             You must set the ID to indicate which CacheDirective you want
-   *             to modify.
+   * @param info Information about the directive to modify. You must set the ID
+   *          to indicate which CacheDirective you want to modify.
+   * @param flags {@link CacheFlag}s to use for this operation.
    * @throws IOException if the directive could not be modified
    */
-  public void modifyCacheDirective(CacheDirectiveInfo info)
-      throws IOException {
-    dfs.modifyCacheDirective(info);
+  public void modifyCacheDirective(CacheDirectiveInfo info,
+      EnumSet<CacheFlag> flags) throws IOException {
+    dfs.modifyCacheDirective(info, flags);
   }
 
   /**

+ 19 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java

@@ -19,7 +19,9 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.util.Iterator;
 import java.util.List;
+import java.util.Random;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -250,33 +252,28 @@ public class BlockListAsLongs implements Iterable<Block> {
   }
 
   /**
-   * The block-id of the indexTh block
-   * @param index - the block whose block-id is desired
-   * @return the block-id
+   * Corrupt the generation stamp of the block with the given index.
+   * Not meant to be used outside of tests.
    */
-  @Deprecated
-  public long getBlockId(final int index)  {
-    return blockId(index);
-  }
-  
-  /**
-   * The block-len of the indexTh block
-   * @param index - the block whose block-len is desired
-   * @return - the block-len
-   */
-  @Deprecated
-  public long getBlockLen(final int index)  {
-    return blockLength(index);
+  @VisibleForTesting
+  public long corruptBlockGSForTesting(final int blockIndex, Random rand) {
+    long oldGS = blockList[index2BlockId(blockIndex) + 2];
+    while (blockList[index2BlockId(blockIndex) + 2] == oldGS) {
+      blockList[index2BlockId(blockIndex) + 2] = rand.nextInt();
+    }
+    return oldGS;
   }
 
   /**
-   * The generation stamp of the indexTh block
-   * @param index - the block whose block-len is desired
-   * @return - the generation stamp
+   * Corrupt the length of the block with the given index by truncation.
+   * Not meant to be used outside of tests.
    */
-  @Deprecated
-  public long getBlockGenStamp(final int index)  {
-    return blockGenerationStamp(index);
+  @VisibleForTesting
+  public long corruptBlockLengthForTesting(final int blockIndex, Random rand) {
+    long oldLength = blockList[index2BlockId(blockIndex) + 1];
+    blockList[index2BlockId(blockIndex) + 1] =
+        rand.nextInt((int) oldLength - 1);
+    return oldLength;
   }
   
   /**

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java

@@ -52,6 +52,14 @@ public final class CacheDirective implements IntrusiveCollection.Element {
   private Element prev;
   private Element next;
 
+  public CacheDirective(CacheDirectiveInfo info) {
+    this(
+        info.getId(),
+        info.getPath().toUri().getPath(),
+        info.getReplication(),
+        info.getExpiration().getAbsoluteMillis());
+  }
+
   public CacheDirective(long id, String path,
       short replication, long expiryTime) {
     Preconditions.checkArgument(id > 0);

+ 25 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSUtil;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Describes a path-based cache directive.
  */
@@ -138,11 +140,22 @@ public class CacheDirectiveInfo {
    */
   public static class Expiration {
 
-    /** Denotes a CacheDirectiveInfo that never expires **/
-    public static final int EXPIRY_NEVER = -1;
+    /**
+     * The maximum value we accept for a relative expiry.
+     */
+    public static final long MAX_RELATIVE_EXPIRY_MS =
+        Long.MAX_VALUE / 4; // This helps prevent weird overflow bugs
+
+    /**
+     * An relative Expiration that never expires.
+     */
+    public static final Expiration NEVER = newRelative(MAX_RELATIVE_EXPIRY_MS);
 
     /**
      * Create a new relative Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
      * 
      * @param ms how long until the CacheDirective expires, in milliseconds
      * @return A relative Expiration
@@ -153,6 +166,9 @@ public class CacheDirectiveInfo {
 
     /**
      * Create a new absolute Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
      * 
      * @param date when the CacheDirective expires
      * @return An absolute Expiration
@@ -163,6 +179,9 @@ public class CacheDirectiveInfo {
 
     /**
      * Create a new absolute Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
      * 
      * @param ms when the CacheDirective expires, in milliseconds since the Unix
      *          epoch.
@@ -176,6 +195,10 @@ public class CacheDirectiveInfo {
     private final boolean isRelative;
 
     private Expiration(long ms, boolean isRelative) {
+      if (isRelative) {
+        Preconditions.checkArgument(ms <= MAX_RELATIVE_EXPIRY_MS,
+            "Expiration time is too far in the future!");
+      }
       this.ms = ms;
       this.isRelative = isRelative;
     }

+ 81 - 84
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hdfs.protocol;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 
 import javax.annotation.Nullable;
@@ -32,14 +30,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
-import org.apache.hadoop.hdfs.util.XMLUtils;
-import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
-import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
-import org.apache.hadoop.io.Text;
-import org.xml.sax.ContentHandler;
-import org.xml.sax.SAXException;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
 
 /**
  * CachePoolInfo describes a cache pool.
@@ -52,6 +43,20 @@ import org.xml.sax.SAXException;
 public class CachePoolInfo {
   public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
 
+  /**
+   * Indicates that the pool does not have a maximum relative expiry.
+   */
+  public static final long RELATIVE_EXPIRY_NEVER =
+      Expiration.MAX_RELATIVE_EXPIRY_MS;
+  /**
+   * Default max relative expiry for cache pools.
+   */
+  public static final long DEFAULT_MAX_RELATIVE_EXPIRY =
+      RELATIVE_EXPIRY_NEVER;
+
+  public static final long LIMIT_UNLIMITED = Long.MAX_VALUE;
+  public static final long DEFAULT_LIMIT = LIMIT_UNLIMITED;
+
   final String poolName;
 
   @Nullable
@@ -64,16 +69,26 @@ public class CachePoolInfo {
   FsPermission mode;
 
   @Nullable
-  Integer weight;
+  Long limit;
+
+  @Nullable
+  Long maxRelativeExpiryMs;
 
   public CachePoolInfo(String poolName) {
     this.poolName = poolName;
   }
-  
+
+  /**
+   * @return Name of the pool.
+   */
   public String getPoolName() {
     return poolName;
   }
 
+  /**
+   * @return The owner of the pool. Along with the group and mode, determines
+   *         who has access to view and modify the pool.
+   */
   public String getOwnerName() {
     return ownerName;
   }
@@ -83,6 +98,10 @@ public class CachePoolInfo {
     return this;
   }
 
+  /**
+   * @return The group of the pool. Along with the owner and mode, determines
+   *         who has access to view and modify the pool.
+   */
   public String getGroupName() {
     return groupName;
   }
@@ -91,7 +110,11 @@ public class CachePoolInfo {
     this.groupName = groupName;
     return this;
   }
-  
+
+  /**
+   * @return Unix-style permissions of the pool. Along with the owner and group,
+   *         determines who has access to view and modify the pool.
+   */
   public FsPermission getMode() {
     return mode;
   }
@@ -101,12 +124,36 @@ public class CachePoolInfo {
     return this;
   }
 
-  public Integer getWeight() {
-    return weight;
+  /**
+   * @return The maximum aggregate number of bytes that can be cached by
+   *         directives in this pool.
+   */
+  public Long getLimit() {
+    return limit;
   }
 
-  public CachePoolInfo setWeight(Integer weight) {
-    this.weight = weight;
+  public CachePoolInfo setLimit(Long bytes) {
+    this.limit = bytes;
+    return this;
+  }
+
+  /**
+   * @return The maximum relative expiration of directives of this pool in
+   *         milliseconds
+   */
+  public Long getMaxRelativeExpiryMs() {
+    return maxRelativeExpiryMs;
+  }
+
+  /**
+   * Set the maximum relative expiration of directives of this pool in
+   * milliseconds.
+   * 
+   * @param ms in milliseconds
+   * @return This builder, for call chaining.
+   */
+  public CachePoolInfo setMaxRelativeExpiryMs(Long ms) {
+    this.maxRelativeExpiryMs = ms;
     return this;
   }
 
@@ -117,7 +164,8 @@ public class CachePoolInfo {
       append(", groupName:").append(groupName).
       append(", mode:").append((mode == null) ? "null" :
           String.format("0%03o", mode.toShort())).
-      append(", weight:").append(weight).
+      append(", limit:").append(limit).
+      append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
       append("}").toString();
   }
   
@@ -134,7 +182,8 @@ public class CachePoolInfo {
         append(ownerName, other.ownerName).
         append(groupName, other.groupName).
         append(mode, other.mode).
-        append(weight, other.weight).
+        append(limit, other.limit).
+        append(maxRelativeExpiryMs, other.maxRelativeExpiryMs).
         isEquals();
   }
 
@@ -145,7 +194,8 @@ public class CachePoolInfo {
         append(ownerName).
         append(groupName).
         append(mode).
-        append(weight).
+        append(limit).
+        append(maxRelativeExpiryMs).
         hashCode();
   }
 
@@ -153,8 +203,17 @@ public class CachePoolInfo {
     if (info == null) {
       throw new InvalidRequestException("CachePoolInfo is null");
     }
-    if ((info.getWeight() != null) && (info.getWeight() < 0)) {
-      throw new InvalidRequestException("CachePool weight is negative.");
+    if ((info.getLimit() != null) && (info.getLimit() < 0)) {
+      throw new InvalidRequestException("Limit is negative.");
+    }
+    if (info.getMaxRelativeExpiryMs() != null) {
+      long maxRelativeExpiryMs = info.getMaxRelativeExpiryMs();
+      if (maxRelativeExpiryMs < 0l) {
+        throw new InvalidRequestException("Max relative expiry is negative.");
+      }
+      if (maxRelativeExpiryMs > Expiration.MAX_RELATIVE_EXPIRY_MS) {
+        throw new InvalidRequestException("Max relative expiry is too big.");
+      }
     }
     validateName(info.poolName);
   }
@@ -167,66 +226,4 @@ public class CachePoolInfo {
       throw new IOException("invalid empty cache pool name");
     }
   }
-
-  public static CachePoolInfo readFrom(DataInput in) throws IOException {
-    String poolName = Text.readString(in);
-    CachePoolInfo info = new CachePoolInfo(poolName);
-    if (in.readBoolean()) {
-      info.setOwnerName(Text.readString(in));
-    }
-    if (in.readBoolean())  {
-      info.setGroupName(Text.readString(in));
-    }
-    if (in.readBoolean()) {
-      info.setMode(FsPermission.read(in));
-    }
-    if (in.readBoolean()) {
-      info.setWeight(in.readInt());
-    }
-    return info;
-  }
-
-  public void writeTo(DataOutput out) throws IOException {
-    Text.writeString(out, poolName);
-    boolean hasOwner, hasGroup, hasMode, hasWeight;
-    hasOwner = ownerName != null;
-    hasGroup = groupName != null;
-    hasMode = mode != null;
-    hasWeight = weight != null;
-    out.writeBoolean(hasOwner);
-    if (hasOwner) {
-      Text.writeString(out, ownerName);
-    }
-    out.writeBoolean(hasGroup);
-    if (hasGroup) {
-      Text.writeString(out, groupName);
-    }
-    out.writeBoolean(hasMode);
-    if (hasMode) {
-      mode.write(out);
-    }
-    out.writeBoolean(hasWeight);
-    if (hasWeight) {
-      out.writeInt(weight);
-    }
-  }
-
-  public void writeXmlTo(ContentHandler contentHandler) throws SAXException {
-    XMLUtils.addSaxString(contentHandler, "POOLNAME", poolName);
-    PermissionStatus perm = new PermissionStatus(ownerName,
-        groupName, mode);
-    FSEditLogOp.permissionStatusToXml(contentHandler, perm);
-    XMLUtils.addSaxString(contentHandler, "WEIGHT", Integer.toString(weight));
-  }
-
-  public static CachePoolInfo readXmlFrom(Stanza st) throws InvalidXmlException {
-    String poolName = st.getValue("POOLNAME");
-    PermissionStatus perm = FSEditLogOp.permissionStatusFromXml(st);
-    int weight = Integer.parseInt(st.getValue("WEIGHT"));
-    return new CachePoolInfo(poolName).
-        setOwnerName(perm.getUserName()).
-        setGroupName(perm.getGroupName()).
-        setMode(perm.getPermission()).
-        setWeight(weight);
-  }
 }

+ 17 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java

@@ -30,6 +30,7 @@ public class CachePoolStats {
   public static class Builder {
     private long bytesNeeded;
     private long bytesCached;
+    private long bytesOverlimit;
     private long filesNeeded;
     private long filesCached;
 
@@ -46,6 +47,11 @@ public class CachePoolStats {
       return this;
     }
 
+    public Builder setBytesOverlimit(long bytesOverlimit) {
+      this.bytesOverlimit = bytesOverlimit;
+      return this;
+    }
+
     public Builder setFilesNeeded(long filesNeeded) {
       this.filesNeeded = filesNeeded;
       return this;
@@ -57,20 +63,22 @@ public class CachePoolStats {
     }
 
     public CachePoolStats build() {
-      return new CachePoolStats(bytesNeeded, bytesCached, filesNeeded,
-          filesCached);
+      return new CachePoolStats(bytesNeeded, bytesCached, bytesOverlimit,
+          filesNeeded, filesCached);
     }
   };
 
   private final long bytesNeeded;
   private final long bytesCached;
+  private final long bytesOverlimit;
   private final long filesNeeded;
   private final long filesCached;
 
-  private CachePoolStats(long bytesNeeded, long bytesCached, long filesNeeded,
-      long filesCached) {
+  private CachePoolStats(long bytesNeeded, long bytesCached,
+      long bytesOverlimit, long filesNeeded, long filesCached) {
     this.bytesNeeded = bytesNeeded;
     this.bytesCached = bytesCached;
+    this.bytesOverlimit = bytesOverlimit;
     this.filesNeeded = filesNeeded;
     this.filesCached = filesCached;
   }
@@ -83,6 +91,10 @@ public class CachePoolStats {
     return bytesCached;
   }
 
+  public long getBytesOverlimit() {
+    return bytesOverlimit;
+  }
+
   public long getFilesNeeded() {
     return filesNeeded;
   }
@@ -95,6 +107,7 @@ public class CachePoolStats {
     return new StringBuilder().append("{").
       append("bytesNeeded:").append(bytesNeeded).
       append(", bytesCached:").append(bytesCached).
+      append(", bytesOverlimit:").append(bytesOverlimit).
       append(", filesNeeded:").append(filesNeeded).
       append(", filesCached:").append(filesCached).
       append("}").toString();

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

@@ -19,9 +19,11 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -354,7 +356,8 @@ public interface ClientProtocol {
    */
   @Idempotent
   public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
-      final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
+      final DatanodeInfo[] existings, final String[] existingStorageIDs,
+      final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
       ) throws AccessControlException, FileNotFoundException,
           SafeModeException, UnresolvedLinkException, IOException;
@@ -983,7 +986,7 @@ public interface ClientProtocol {
    */
   @AtMostOnce
   public void updatePipeline(String clientName, ExtendedBlock oldBlock, 
-      ExtendedBlock newBlock, DatanodeID[] newNodes)
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
       throws IOException;
 
   /**
@@ -1099,23 +1102,24 @@ public interface ClientProtocol {
    * Add a CacheDirective to the CacheManager.
    * 
    * @param directive A CacheDirectiveInfo to be added
+   * @param flags {@link CacheFlag}s to use for this operation.
    * @return A CacheDirectiveInfo associated with the added directive
    * @throws IOException if the directive could not be added
    */
   @AtMostOnce
-  public long addCacheDirective(
-      CacheDirectiveInfo directive) throws IOException;
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
 
   /**
    * Modify a CacheDirective in the CacheManager.
    * 
-   * @return directive The directive to modify.  Must contain 
-   *                   a directive ID.
+   * @return directive The directive to modify. Must contain a directive ID.
+   * @param flags {@link CacheFlag}s to use for this operation.
    * @throws IOException if the directive could not be modified
    */
   @AtMostOnce
-  public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException;
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
 
   /**
    * Remove a CacheDirectiveInfo from the CacheManager.

+ 37 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.hdfs.protocol;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * This class represents the primary identifier for a Datanode.
  * Datanodes are identified by how they can be contacted (hostname
@@ -40,37 +42,46 @@ public class DatanodeID implements Comparable<DatanodeID> {
   private String ipAddr;     // IP address
   private String hostName;   // hostname claimed by datanode
   private String peerHostName; // hostname from the actual connection
-  private String storageID;  // unique per cluster storageID
   private int xferPort;      // data streaming port
   private int infoPort;      // info server port
   private int infoSecurePort; // info server port
   private int ipcPort;       // IPC server port
 
+  /**
+   * UUID identifying a given datanode. For upgraded Datanodes this is the
+   * same as the StorageID that was previously used by this Datanode. 
+   * For newly formatted Datanodes it is a UUID.
+   */
+  private String datanodeUuid = null;
+
   public DatanodeID(DatanodeID from) {
     this(from.getIpAddr(),
         from.getHostName(),
-        from.getStorageID(),
+        from.getDatanodeUuid(),
         from.getXferPort(),
         from.getInfoPort(),
         from.getInfoSecurePort(),
         from.getIpcPort());
     this.peerHostName = from.getPeerHostName();
   }
-  
+
   /**
    * Create a DatanodeID
    * @param ipAddr IP
    * @param hostName hostname
-   * @param storageID data storage ID
+   * @param datanodeUuid data node ID, UUID for new Datanodes, may be the
+   *                     storage ID for pre-UUID datanodes. NULL if unknown
+   *                     e.g. if this is a new datanode. A new UUID will
+   *                     be assigned by the namenode.
    * @param xferPort data transfer port
    * @param infoPort info server port 
    * @param ipcPort ipc server port
    */
-  public DatanodeID(String ipAddr, String hostName, String storageID,
+  public DatanodeID(String ipAddr, String hostName, String datanodeUuid,
       int xferPort, int infoPort, int infoSecurePort, int ipcPort) {
     this.ipAddr = ipAddr;
     this.hostName = hostName;
-    this.storageID = storageID;
+    this.datanodeUuid = checkDatanodeUuid(datanodeUuid);
     this.xferPort = xferPort;
     this.infoPort = infoPort;
     this.infoSecurePort = infoSecurePort;
@@ -85,8 +96,24 @@ public class DatanodeID implements Comparable<DatanodeID> {
     this.peerHostName = peerHostName;
   }
   
-  public void setStorageID(String storageID) {
-    this.storageID = storageID;
+  /**
+   * @return data node ID.
+   */
+  public String getDatanodeUuid() {
+    return datanodeUuid;
+  }
+
+  @VisibleForTesting
+  public void setDatanodeUuidForTesting(String datanodeUuid) {
+    this.datanodeUuid = datanodeUuid;
+  }
+
+  private String checkDatanodeUuid(String uuid) {
+    if (uuid == null || uuid.isEmpty()) {
+      return null;
+    } else {
+      return uuid;
+    }
   }
 
   /**
@@ -168,13 +195,6 @@ public class DatanodeID implements Comparable<DatanodeID> {
     return useHostname ? getIpcAddrWithHostname() : getIpcAddr();
   }
 
-  /**
-   * @return data storage ID.
-   */
-  public String getStorageID() {
-    return storageID;
-  }
-
   /**
    * @return xferPort (the port for data streaming)
    */
@@ -212,12 +232,12 @@ public class DatanodeID implements Comparable<DatanodeID> {
       return false;
     }
     return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) &&
-            storageID.equals(((DatanodeID)to).getStorageID()));
+        datanodeUuid.equals(((DatanodeID)to).getDatanodeUuid()));
   }
   
   @Override
   public int hashCode() {
-    return getXferAddr().hashCode()^ storageID.hashCode();
+    return getXferAddr().hashCode()^ datanodeUuid.hashCode();
   }
   
   @Override

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

@@ -115,7 +115,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long lastUpdate, final int xceiverCount,
       final AdminStates adminState) {
-    this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getStorageID(),
+    this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getDatanodeUuid(),
         nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
         nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
         cacheCapacity, cacheUsed, lastUpdate, xceiverCount, location,
@@ -124,13 +124,13 @@ public class DatanodeInfo extends DatanodeID implements Node {
 
   /** Constructor */
   public DatanodeInfo(final String ipAddr, final String hostName,
-      final String storageID, final int xferPort, final int infoPort,
+      final String datanodeUuid, final int xferPort, final int infoPort,
       final int infoSecurePort, final int ipcPort,
       final long capacity, final long dfsUsed, final long remaining,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long lastUpdate, final int xceiverCount,
       final String networkLocation, final AdminStates adminState) {
-    super(ipAddr, hostName, storageID, xferPort, infoPort,
+    super(ipAddr, hostName, datanodeUuid, xferPort, infoPort,
             infoSecurePort, ipcPort);
     this.capacity = capacity;
     this.dfsUsed = dfsUsed;

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -107,7 +107,10 @@ public class LayoutVersion {
         "block IDs in the edits log and image files"),
     EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to " 
         + "enable rebuilding retry cache in case of HA failover"),
-    CACHING(-48, "Support for cache pools and path-based caching");
+    CACHING(-48, "Support for cache pools and path-based caching"),
+    ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
+        + " Use distinct StorageUuid per storage directory.");
+
     
     final int lv;
     final int ancestorLV;
@@ -248,3 +251,4 @@ public class LayoutVersion {
     throw new AssertionError("All layout versions are reserved.");
   }
 }
+

+ 39 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -21,7 +21,9 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.common.base.Preconditions;
@@ -40,6 +42,10 @@ public class LocatedBlock {
   private ExtendedBlock b;
   private long offset;  // offset of the first byte of the block in the file
   private DatanodeInfo[] locs;
+  /** Storage ID for each replica */
+  private String[] storageIDs;
+  // Storage type for each replica, if reported.
+  private StorageType[] storageTypes;
   // corrupt flag is true if all of the replicas of a block are corrupt.
   // else false. If block has few corrupt replicas, they are filtered and 
   // their locations are not part of this object
@@ -54,20 +60,34 @@ public class LocatedBlock {
   private static final DatanodeInfo[] EMPTY_LOCS = new DatanodeInfo[0];
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
-    this(b, locs, -1); // startOffset is unknown
-  }
-
-  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset) {
-    this(b, locs, startOffset, false);
+    this(b, locs, -1, false); // startOffset is unknown
   }
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset, 
                       boolean corrupt) {
-    this(b, locs, startOffset, corrupt, EMPTY_LOCS);
+    this(b, locs, null, null, startOffset, corrupt, EMPTY_LOCS);
+  }
+
+  public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages) {
+    this(b, storages, -1, false); // startOffset is unknown
   }
 
-  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
-      boolean corrupt, DatanodeInfo[] cachedLocs) {
+  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
+                      String[] storageIDs, StorageType[] storageTypes) {
+    this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
+  }
+
+  public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
+      long startOffset, boolean corrupt) {
+    this(b, DatanodeStorageInfo.toDatanodeInfos(storages),
+        DatanodeStorageInfo.toStorageIDs(storages),
+        DatanodeStorageInfo.toStorageTypes(storages),
+        startOffset, corrupt, EMPTY_LOCS); // startOffset is unknown
+  }
+
+  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
+                      StorageType[] storageTypes, long startOffset,
+                      boolean corrupt, DatanodeInfo[] cachedLocs) {
     this.b = b;
     this.offset = startOffset;
     this.corrupt = corrupt;
@@ -76,6 +96,8 @@ public class LocatedBlock {
     } else {
       this.locs = locs;
     }
+    this.storageIDs = storageIDs;
+    this.storageTypes = storageTypes;
     Preconditions.checkArgument(cachedLocs != null,
         "cachedLocs should not be null, use a different constructor");
     if (cachedLocs.length == 0) {
@@ -100,7 +122,15 @@ public class LocatedBlock {
   public DatanodeInfo[] getLocations() {
     return locs;
   }
+
+  public StorageType[] getStorageTypes() {
+    return storageTypes;
+  }
   
+  public String[] getStorageIDs() {
+    return storageIDs;
+  }
+
   public long getStartOffset() {
     return offset;
   }
@@ -161,3 +191,4 @@ public class LocatedBlock {
         + "}";
   }
 }
+

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnregisteredNodeException.java

@@ -51,7 +51,7 @@ public class UnregisteredNodeException extends IOException {
    */
   public UnregisteredNodeException(DatanodeID nodeID, DatanodeInfo storedNode) {
     super("Data node " + nodeID + " is attempting to report storage ID " 
-          + nodeID.getStorageID() + ". Node " 
+          + nodeID.getDatanodeUuid() + ". Node "
           + storedNode + " is expected to serve this storage.");
   }
 }

+ 19 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -320,7 +320,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     try {
       HdfsFileStatus result = server.create(req.getSrc(),
           PBHelper.convert(req.getMasked()), req.getClientName(),
-          PBHelper.convert(req.getCreateFlag()), req.getCreateParent(),
+          PBHelper.convertCreateFlag(req.getCreateFlag()), req.getCreateParent(),
           (short) req.getReplication(), req.getBlockSize());
 
       if (result != null) {
@@ -425,14 +425,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     try {
       List<DatanodeInfoProto> existingList = req.getExistingsList();
+      List<String> existingStorageIDsList = req.getExistingStorageUuidsList();
       List<DatanodeInfoProto> excludesList = req.getExcludesList();
-      LocatedBlock result = server.getAdditionalDatanode(
-          req.getSrc(), PBHelper.convert(req.getBlk()),
+      LocatedBlock result = server.getAdditionalDatanode(req.getSrc(),
+          PBHelper.convert(req.getBlk()),
           PBHelper.convert(existingList.toArray(
               new DatanodeInfoProto[existingList.size()])),
+          existingStorageIDsList.toArray(
+              new String[existingStorageIDsList.size()]),
           PBHelper.convert(excludesList.toArray(
               new DatanodeInfoProto[excludesList.size()])), 
-              req.getNumAdditionalNodes(), req.getClientName());
+          req.getNumAdditionalNodes(), req.getClientName());
       return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
           PBHelper.convert(result))
           .build();
@@ -833,10 +836,12 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       UpdatePipelineRequestProto req) throws ServiceException {
     try {
       List<DatanodeIDProto> newNodes = req.getNewNodesList();
-      server
-          .updatePipeline(req.getClientName(), PBHelper.convert(req
-              .getOldBlock()), PBHelper.convert(req.getNewBlock()), PBHelper
-              .convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])));
+      List<String> newStorageIDs = req.getStorageIDsList();
+      server.updatePipeline(req.getClientName(),
+          PBHelper.convert(req.getOldBlock()),
+          PBHelper.convert(req.getNewBlock()),
+          PBHelper.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])),
+          newStorageIDs.toArray(new String[newStorageIDs.size()]));
       return VOID_UPDATEPIPELINE_RESPONSE;
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1029,9 +1034,11 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, AddCacheDirectiveRequestProto request)
       throws ServiceException {
     try {
+      long id = server.addCacheDirective(
+          PBHelper.convert(request.getInfo()),
+          PBHelper.convertCacheFlags(request.getCacheFlags()));
       return AddCacheDirectiveResponseProto.newBuilder().
-              setId(server.addCacheDirective(
-                  PBHelper.convert(request.getInfo()))).build();
+              setId(id).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1043,7 +1050,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     try {
       server.modifyCacheDirective(
-          PBHelper.convert(request.getInfo()));
+          PBHelper.convert(request.getInfo()),
+          PBHelper.convertCacheFlags(request.getCacheFlags()));
       return ModifyCacheDirectiveResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);

+ 24 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -21,10 +21,12 @@ import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -351,7 +353,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
-      DatanodeInfo[] existings, DatanodeInfo[] excludes,
+      DatanodeInfo[] existings, String[] existingStorageIDs,
+      DatanodeInfo[] excludes,
       int numAdditionalNodes, String clientName) throws AccessControlException,
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
       IOException {
@@ -360,6 +363,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setSrc(src)
         .setBlk(PBHelper.convert(blk))
         .addAllExistings(PBHelper.convert(existings))
+        .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
         .addAllExcludes(PBHelper.convert(excludes))
         .setNumAdditionalNodes(numAdditionalNodes)
         .setClientName(clientName)
@@ -796,12 +800,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
-      ExtendedBlock newBlock, DatanodeID[] newNodes) throws IOException {
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
     UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
         .setClientName(clientName)
         .setOldBlock(PBHelper.convert(oldBlock))
         .setNewBlock(PBHelper.convert(newBlock))
         .addAllNewNodes(Arrays.asList(PBHelper.convert(newNodes)))
+        .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs))
         .build();
     try {
       rpcProxy.updatePipeline(null, req);
@@ -1000,24 +1005,32 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public long addCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
     try {
-      return rpcProxy.addCacheDirective(null, 
-              AddCacheDirectiveRequestProto.newBuilder().
-                  setInfo(PBHelper.convert(directive)).build()).getId();
+      AddCacheDirectiveRequestProto.Builder builder =
+          AddCacheDirectiveRequestProto.newBuilder().
+              setInfo(PBHelper.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelper.convertCacheFlags(flags));
+      }
+      return rpcProxy.addCacheDirective(null, builder.build()).getId();
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
 
   @Override
-  public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
     try {
-      rpcProxy.modifyCacheDirective(null,
+      ModifyCacheDirectiveRequestProto.Builder builder =
           ModifyCacheDirectiveRequestProto.newBuilder().
-              setInfo(PBHelper.convert(directive)).build());
+              setInfo(PBHelper.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelper.convertCacheFlags(flags));
+      }
+      rpcProxy.modifyCacheDirective(null, builder.build());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

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

@@ -245,7 +245,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     for (StorageReceivedDeletedBlocks storageBlock : receivedAndDeletedBlocks) {
       StorageReceivedDeletedBlocksProto.Builder repBuilder = 
           StorageReceivedDeletedBlocksProto.newBuilder();
-      repBuilder.setStorageID(storageBlock.getStorageID());
+      repBuilder.setStorageUuid(storageBlock.getStorageID());
       for (ReceivedDeletedBlockInfo rdBlock : storageBlock.getBlocks()) {
         repBuilder.addBlocks(PBHelper.convert(rdBlock));
       }

+ 3 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlo
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
@@ -102,14 +101,8 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       HeartbeatRequestProto request) throws ServiceException {
     HeartbeatResponse response;
     try {
-      List<StorageReportProto> list = request.getReportsList();
-      StorageReport[] report = new StorageReport[list.size()];
-      int i = 0;
-      for (StorageReportProto p : list) {
-        report[i++] = new StorageReport(p.getStorageID(), p.getFailed(),
-            p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
-            p.getBlockPoolUsed());
-      }
+      final StorageReport[] report = PBHelper.convertStorageReports(
+          request.getReportsList());
       response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
           report, request.getCacheCapacity(), request.getCacheUsed(),
           request.getXmitsInProgress(),
@@ -198,7 +191,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       for (int j = 0; j < list.size(); j++) {
         rdBlocks[j] = PBHelper.convert(list.get(j));
       }
-      info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageID(), rdBlocks);
+      info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageUuid(), rdBlocks);
     }
     try {
       impl.blockReceivedAndDeleted(PBHelper.convert(request.getRegistration()),

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

@@ -82,6 +82,6 @@ public class InterDatanodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
     return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
-        .setStorageID(storageID).build();
+        .setStorageUuid(storageID).build();
   }
 }

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

@@ -109,7 +109,7 @@ public class InterDatanodeProtocolTranslatorPB implements
         .setNewLength(newLength).setRecoveryId(recoveryId).build();
     try {
       return rpcProxy.updateReplicaUnderRecovery(NULL_CONTROLLER, req
-          ).getStorageID();
+          ).getStorageUuid();
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

+ 172 - 43
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -27,6 +27,7 @@ import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -35,6 +36,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
@@ -52,17 +54,18 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
@@ -122,6 +125,8 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -242,17 +247,20 @@ public class PBHelper {
 
   // DatanodeId
   public static DatanodeID convert(DatanodeIDProto dn) {
-    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getStorageID(),
+    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
         dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
         .getInfoSecurePort() : 0, dn.getIpcPort());
   }
 
   public static DatanodeIDProto convert(DatanodeID dn) {
+    // For wire compatibility with older versions we transmit the StorageID
+    // which is the same as the DatanodeUuid. Since StorageID is a required
+    // field we pass the empty string if the DatanodeUuid is not yet known.
     return DatanodeIDProto.newBuilder()
         .setIpAddr(dn.getIpAddr())
         .setHostName(dn.getHostName())
-        .setStorageID(dn.getStorageID())
         .setXferPort(dn.getXferPort())
+        .setDatanodeUuid(dn.getDatanodeUuid() != null ? dn.getDatanodeUuid() : "")
         .setInfoPort(dn.getInfoPort())
         .setInfoSecurePort(dn.getInfoSecurePort())
         .setIpcPort(dn.getIpcPort()).build();
@@ -294,12 +302,16 @@ public class PBHelper {
   public static BlockWithLocationsProto convert(BlockWithLocations blk) {
     return BlockWithLocationsProto.newBuilder()
         .setBlock(convert(blk.getBlock()))
-        .addAllStorageIDs(Arrays.asList(blk.getStorageIDs())).build();
+        .addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids()))
+        .addAllStorageUuids(Arrays.asList(blk.getStorageIDs())).build();
   }
 
   public static BlockWithLocations convert(BlockWithLocationsProto b) {
-    return new BlockWithLocations(convert(b.getBlock()), b.getStorageIDsList()
-        .toArray(new String[0]));
+    final List<String> datanodeUuids = b.getDatanodeUuidsList();
+    final List<String> storageUuids = b.getStorageUuidsList();
+    return new BlockWithLocations(convert(b.getBlock()),
+        datanodeUuids.toArray(new String[datanodeUuids.size()]),
+        storageUuids.toArray(new String[storageUuids.size()]));
   }
 
   public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
@@ -499,21 +511,7 @@ public class PBHelper {
   
   static public DatanodeInfoProto convertDatanodeInfo(DatanodeInfo di) {
     if (di == null) return null;
-    DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
-    if (di.getNetworkLocation() != null) {
-      builder.setLocation(di.getNetworkLocation());
-    }
-        
-    return builder.
-     setId(PBHelper.convert((DatanodeID) di)).
-     setCapacity(di.getCapacity()).
-     setDfsUsed(di.getDfsUsed()).
-     setRemaining(di.getRemaining()).
-     setBlockPoolUsed(di.getBlockPoolUsed()).
-     setLastUpdate(di.getLastUpdate()).
-     setXceiverCount(di.getXceiverCount()).
-     setAdminState(PBHelper.convert(di.getAdminState())).
-     build();     
+    return convert(di);
   }
   
   
@@ -557,15 +555,20 @@ public class PBHelper {
   
   public static DatanodeInfoProto convert(DatanodeInfo info) {
     DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
-    builder.setBlockPoolUsed(info.getBlockPoolUsed());
-    builder.setAdminState(PBHelper.convert(info.getAdminState()));
-    builder.setCapacity(info.getCapacity())
-        .setDfsUsed(info.getDfsUsed())
+    if (info.getNetworkLocation() != null) {
+      builder.setLocation(info.getNetworkLocation());
+    }
+    builder
         .setId(PBHelper.convert((DatanodeID)info))
-        .setLastUpdate(info.getLastUpdate())
-        .setLocation(info.getNetworkLocation())
+        .setCapacity(info.getCapacity())
+        .setDfsUsed(info.getDfsUsed())
         .setRemaining(info.getRemaining())
+        .setBlockPoolUsed(info.getBlockPoolUsed())
+        .setCacheCapacity(info.getCacheCapacity())
+        .setCacheUsed(info.getCacheUsed())
+        .setLastUpdate(info.getLastUpdate())
         .setXceiverCount(info.getXceiverCount())
+        .setAdminState(PBHelper.convert(info.getAdminState()))
         .build();
     return builder.build();
   }
@@ -601,6 +604,17 @@ public class PBHelper {
         "Found additional cached replica locations that are not in the set of"
         + " storage-backed locations!");
 
+    StorageType[] storageTypes = b.getStorageTypes();
+    if (storageTypes != null) {
+      for (int i = 0; i < storageTypes.length; ++i) {
+        builder.addStorageTypes(PBHelper.convertStorageType(storageTypes[i]));
+      }
+    }
+    final String[] storageIDs = b.getStorageIDs();
+    if (storageIDs != null) {
+      builder.addAllStorageIDs(Arrays.asList(storageIDs));
+    }
+
     return builder.setB(PBHelper.convert(b.getBlock()))
         .setBlockToken(PBHelper.convert(b.getBlockToken()))
         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
@@ -613,6 +627,25 @@ public class PBHelper {
     for (int i = 0; i < locs.size(); i++) {
       targets[i] = PBHelper.convert(locs.get(i));
     }
+
+    final int storageTypesCount = proto.getStorageTypesCount();
+    final StorageType[] storageTypes;
+    if (storageTypesCount == 0) {
+      storageTypes = null;
+    } else {
+      Preconditions.checkState(storageTypesCount == locs.size());
+      storageTypes = convertStorageTypeProtos(proto.getStorageTypesList());
+    }
+
+    final int storageIDsCount = proto.getStorageIDsCount();
+    final String[] storageIDs;
+    if (storageIDsCount == 0) {
+      storageIDs = null;
+    } else {
+      Preconditions.checkState(storageIDsCount == locs.size());
+      storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
+    }
+
     // Set values from the isCached list, re-using references from loc
     List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
     List<Boolean> isCachedList = proto.getIsCachedList();
@@ -623,7 +656,7 @@ public class PBHelper {
     }
 
     LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
-        proto.getOffset(), proto.getCorrupt(),
+        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
         cachedLocs.toArray(new DatanodeInfo[0]));
     lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
 
@@ -766,7 +799,8 @@ public class PBHelper {
     for (int i = 0; i < blocks.length; i++) {
       builder.addBlocks(PBHelper.convert(blocks[i]));
     }
-    builder.addAllTargets(PBHelper.convert(cmd.getTargets()));
+    builder.addAllTargets(convert(cmd.getTargets()))
+           .addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs()));
     return builder.build();
   }
   
@@ -799,6 +833,15 @@ public class PBHelper {
     return Arrays.asList(ret);
   }
 
+  private static List<StorageUuidsProto> convert(String[][] targetStorageUuids) {
+    StorageUuidsProto[] ret = new StorageUuidsProto[targetStorageUuids.length];
+    for (int i = 0; i < targetStorageUuids.length; i++) {
+      ret[i] = StorageUuidsProto.newBuilder()
+          .addAllStorageUuids(Arrays.asList(targetStorageUuids[i])).build();
+    }
+    return Arrays.asList(ret);
+  }
+
   public static DatanodeCommandProto convert(DatanodeCommand datanodeCommand) {
     DatanodeCommandProto.Builder builder = DatanodeCommandProto.newBuilder();
     if (datanodeCommand == null) {
@@ -878,6 +921,14 @@ public class PBHelper {
     for (int i = 0; i < targetList.size(); i++) {
       targets[i] = PBHelper.convert(targetList.get(i));
     }
+
+    List<StorageUuidsProto> targetStorageUuidsList = blkCmd.getTargetStorageUuidsList();
+    String[][] targetStorageIDs = new String[targetStorageUuidsList.size()][];
+    for(int i = 0; i < targetStorageIDs.length; i++) {
+      List<String> storageIDs = targetStorageUuidsList.get(i).getStorageUuidsList();
+      targetStorageIDs[i] = storageIDs.toArray(new String[storageIDs.size()]);
+    }
+
     int action = DatanodeProtocol.DNA_UNKNOWN;
     switch (blkCmd.getAction()) {
     case TRANSFER:
@@ -892,7 +943,8 @@ public class PBHelper {
     default:
       throw new AssertionError("Unknown action type: " + blkCmd.getAction());
     }
-    return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets);
+    return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets,
+        targetStorageIDs);
   }
 
   public static BlockIdCommand convert(BlockIdCommandProto blkIdCmd) {
@@ -1123,7 +1175,7 @@ public class PBHelper {
     return value;
   }
   
-  public static EnumSetWritable<CreateFlag> convert(int flag) {
+  public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
     EnumSet<CreateFlag> result = 
        EnumSet.noneOf(CreateFlag.class);   
     if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
@@ -1138,7 +1190,23 @@ public class PBHelper {
     }
     return new EnumSetWritable<CreateFlag>(result);
   }
-  
+
+  public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
+    int value = 0;
+    if (flags.contains(CacheFlag.FORCE)) {
+      value |= CacheFlagProto.FORCE.getNumber();
+    }
+    return value;
+  }
+
+  public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
+    EnumSet<CacheFlag> result = EnumSet.noneOf(CacheFlag.class);
+    if ((flags & CacheFlagProto.FORCE_VALUE) == CacheFlagProto.FORCE_VALUE) {
+      result.add(CacheFlag.FORCE);
+    }
+    return result;
+  }
+
   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
     if (fs == null)
       return null;
@@ -1422,11 +1490,12 @@ public class PBHelper {
 
   public static DatanodeStorageProto convert(DatanodeStorage s) {
     return DatanodeStorageProto.newBuilder()
-        .setState(PBHelper.convert(s.getState()))
-        .setStorageID(s.getStorageID()).build();
+        .setState(PBHelper.convertState(s.getState()))
+        .setStorageType(PBHelper.convertStorageType(s.getStorageType()))
+        .setStorageUuid(s.getStorageID()).build();
   }
 
-  private static StorageState convert(State state) {
+  private static StorageState convertState(State state) {
     switch(state) {
     case READ_ONLY:
       return StorageState.READ_ONLY;
@@ -1436,11 +1505,26 @@ public class PBHelper {
     }
   }
 
+  private static StorageTypeProto convertStorageType(
+      StorageType type) {
+    switch(type) {
+    case DISK:
+      return StorageTypeProto.DISK;
+    case SSD:
+      return StorageTypeProto.SSD;
+    default:
+      throw new IllegalStateException(
+          "BUG: StorageType not found, type=" + type);
+    }
+  }
+
   public static DatanodeStorage convert(DatanodeStorageProto s) {
-    return new DatanodeStorage(s.getStorageID(), PBHelper.convert(s.getState()));
+    return new DatanodeStorage(s.getStorageUuid(),
+                               PBHelper.convertState(s.getState()),
+                               PBHelper.convertType(s.getStorageType()));
   }
 
-  private static State convert(StorageState state) {
+  private static State convertState(StorageState state) {
     switch(state) {
     case READ_ONLY:
       return DatanodeStorage.State.READ_ONLY;
@@ -1450,14 +1534,50 @@ public class PBHelper {
     }
   }
 
+  private static StorageType convertType(StorageTypeProto type) {
+    switch(type) {
+      case DISK:
+        return StorageType.DISK;
+      case SSD:
+        return StorageType.SSD;
+      default:
+        throw new IllegalStateException(
+            "BUG: StorageTypeProto not found, type=" + type);
+    }
+  }
+
+  private static StorageType[] convertStorageTypeProtos(
+      List<StorageTypeProto> storageTypesList) {
+    final StorageType[] storageTypes = new StorageType[storageTypesList.size()];
+    for (int i = 0; i < storageTypes.length; ++i) {
+      storageTypes[i] = PBHelper.convertType(storageTypesList.get(i));
+    }
+    return storageTypes;
+  }
+
   public static StorageReportProto convert(StorageReport r) {
     StorageReportProto.Builder builder = StorageReportProto.newBuilder()
         .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
         .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
-        .setStorageID(r.getStorageID());
+        .setStorageUuid(r.getStorageID());
     return builder.build();
   }
 
+  public static StorageReport convert(StorageReportProto p) {
+    return new StorageReport(p.getStorageUuid(), p.getFailed(),
+        p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
+        p.getBlockPoolUsed());
+  }
+
+  public static StorageReport[] convertStorageReports(
+      List<StorageReportProto> list) {
+    final StorageReport[] report = new StorageReport[list.size()];
+    for (int i = 0; i < report.length; i++) {
+      report[i] = convert(list.get(i));
+    }
+    return report;
+  }
+
   public static JournalInfo convert(JournalInfoProto info) {
     int lv = info.hasLayoutVersion() ? info.getLayoutVersion() : 0;
     int nsID = info.hasNamespaceID() ? info.getNamespaceID() : 0;
@@ -1684,8 +1804,11 @@ public class PBHelper {
     if (info.getMode() != null) {
       builder.setMode(info.getMode().toShort());
     }
-    if (info.getWeight() != null) {
-      builder.setWeight(info.getWeight());
+    if (info.getLimit() != null) {
+      builder.setLimit(info.getLimit());
+    }
+    if (info.getMaxRelativeExpiryMs() != null) {
+      builder.setMaxRelativeExpiry(info.getMaxRelativeExpiryMs());
     }
     return builder.build();
   }
@@ -1703,8 +1826,11 @@ public class PBHelper {
     if (proto.hasMode()) {
       info.setMode(new FsPermission((short)proto.getMode()));
     }
-    if (proto.hasWeight()) {
-      info.setWeight(proto.getWeight());
+    if (proto.hasLimit())  {
+      info.setLimit(proto.getLimit());
+    }
+    if (proto.hasMaxRelativeExpiry()) {
+      info.setMaxRelativeExpiryMs(proto.getMaxRelativeExpiry());
     }
     return info;
   }
@@ -1713,6 +1839,7 @@ public class PBHelper {
     CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
     builder.setBytesNeeded(stats.getBytesNeeded());
     builder.setBytesCached(stats.getBytesCached());
+    builder.setBytesOverlimit(stats.getBytesOverlimit());
     builder.setFilesNeeded(stats.getFilesNeeded());
     builder.setFilesCached(stats.getFilesCached());
     return builder.build();
@@ -1722,6 +1849,7 @@ public class PBHelper {
     CachePoolStats.Builder builder = new CachePoolStats.Builder();
     builder.setBytesNeeded(proto.getBytesNeeded());
     builder.setBytesCached(proto.getBytesCached());
+    builder.setBytesOverlimit(proto.getBytesOverlimit());
     builder.setFilesNeeded(proto.getFilesNeeded());
     builder.setFilesCached(proto.getFilesCached());
     return builder.build();
@@ -1756,3 +1884,4 @@ public class PBHelper {
     return new ExactSizeInputStream(input, size);
   }
 }
+

+ 25 - 48
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.balancer;
 
 import static com.google.common.base.Preconditions.checkArgument;
-
 import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
 
 import java.io.BufferedInputStream;
@@ -221,9 +220,9 @@ public class Balancer {
   private Map<Block, BalancerBlock> globalBlockList
                  = new HashMap<Block, BalancerBlock>();
   private MovedBlocks movedBlocks = new MovedBlocks();
-  // Map storage IDs to BalancerDatanodes
-  private Map<String, BalancerDatanode> datanodes
-                 = new HashMap<String, BalancerDatanode>();
+  /** Map (datanodeUuid -> BalancerDatanodes) */
+  private final Map<String, BalancerDatanode> datanodeMap
+      = new HashMap<String, BalancerDatanode>();
   
   private NetworkTopology cluster;
 
@@ -241,6 +240,14 @@ public class Balancer {
     private PendingBlockMove() {
     }
     
+    @Override
+    public String toString() {
+      final Block b = block.getBlock();
+      return b + " with size=" + b.getNumBytes() + " from "
+          + source.getDisplayName() + " to " + target.getDisplayName()
+          + " through " + proxySource.getDisplayName();
+    }
+
     /* choose a block & a proxy source for this pendingMove 
      * whose source & target have already been chosen.
      * 
@@ -272,11 +279,7 @@ public class Balancer {
             if ( chooseProxySource() ) {
               movedBlocks.add(block);
               if (LOG.isDebugEnabled()) {
-                LOG.debug("Decided to move block "+ block.getBlockId()
-                    +" with a length of "+StringUtils.byteDesc(block.getNumBytes())
-                    + " bytes from " + source.getDisplayName()
-                    + " to " + target.getDisplayName()
-                    + " using proxy source " + proxySource.getDisplayName() );
+                LOG.debug("Decided to move " + this);
               }
               return true;
             }
@@ -353,17 +356,9 @@ public class Balancer {
         sendRequest(out);
         receiveResponse(in);
         bytesMoved.inc(block.getNumBytes());
-        LOG.info( "Moving block " + block.getBlock().getBlockId() +
-              " from "+ source.getDisplayName() + " to " +
-              target.getDisplayName() + " through " +
-              proxySource.getDisplayName() +
-              " is succeeded." );
+        LOG.info("Successfully moved " + this);
       } catch (IOException e) {
-        LOG.warn("Error moving block "+block.getBlockId()+
-            " from " + source.getDisplayName() + " to " +
-            target.getDisplayName() + " through " +
-            proxySource.getDisplayName() +
-            ": "+e.getMessage());
+        LOG.warn("Failed to move " + this + ": " + e.getMessage());
       } finally {
         IOUtils.closeStream(out);
         IOUtils.closeStream(in);
@@ -415,9 +410,7 @@ public class Balancer {
         @Override
         public void run() {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Starting moving "+ block.getBlockId() +
-                " from " + proxySource.getDisplayName() + " to " +
-                target.getDisplayName());
+            LOG.debug("Start moving " + PendingBlockMove.this);
           }
           dispatch();
         }
@@ -464,11 +457,6 @@ public class Balancer {
       return block;
     }
     
-    /* Return the block id */
-    private long getBlockId() {
-      return block.getBlockId();
-    }
-    
     /* Return the length of the block */
     private long getNumBytes() {
       return block.getNumBytes();
@@ -552,7 +540,7 @@ public class Balancer {
     
     /* Get the storage id of the datanode */
     protected String getStorageID() {
-      return datanode.getStorageID();
+      return datanode.getDatanodeUuid();
     }
     
     /** Decide if still need to move more bytes */
@@ -675,10 +663,10 @@ public class Balancer {
         
           synchronized (block) {
             // update locations
-            for ( String storageID : blk.getStorageIDs() ) {
-              BalancerDatanode datanode = datanodes.get(storageID);
+            for (String datanodeUuid : blk.getDatanodeUuids()) {
+              final BalancerDatanode d = datanodeMap.get(datanodeUuid);
               if (datanode != null) { // not an unknown datanode
-                block.addLocation(datanode);
+                block.addLocation(d);
               }
             }
           }
@@ -852,16 +840,6 @@ public class Balancer {
                         DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_DEFAULT));
   }
   
-  /* Shuffle datanode array */
-  static private void shuffleArray(DatanodeInfo[] datanodes) {
-    for (int i=datanodes.length; i>1; i--) {
-      int randomIndex = DFSUtil.getRandom().nextInt(i);
-      DatanodeInfo tmp = datanodes[randomIndex];
-      datanodes[randomIndex] = datanodes[i-1];
-      datanodes[i-1] = tmp;
-    }
-  }
-  
   /* Given a data node set, build a network topology and decide
    * over-utilized datanodes, above average utilized datanodes, 
    * below average utilized datanodes, and underutilized datanodes. 
@@ -891,8 +869,7 @@ public class Balancer {
      * an increasing order or a decreasing order.
      */  
     long overLoadedBytes = 0L, underLoadedBytes = 0L;
-    shuffleArray(datanodes);
-    for (DatanodeInfo datanode : datanodes) {
+    for (DatanodeInfo datanode : DFSUtil.shuffle(datanodes)) {
       if (datanode.isDecommissioned() || datanode.isDecommissionInProgress()) {
         continue; // ignore decommissioning or decommissioned nodes
       }
@@ -923,13 +900,13 @@ public class Balancer {
               datanodeS.utilization)*datanodeS.datanode.getCapacity()/100.0);
         }
       }
-      this.datanodes.put(datanode.getStorageID(), datanodeS);
+      datanodeMap.put(datanode.getDatanodeUuid(), datanodeS);
     }
 
     //logging
     logNodes();
     
-    assert (this.datanodes.size() == 
+    assert (this.datanodeMap.size() == 
       overUtilizedDatanodes.size()+underUtilizedDatanodes.size()+
       aboveAvgUtilizedDatanodes.size()+belowAvgUtilizedDatanodes.size())
       : "Mismatched number of datanodes";
@@ -1001,9 +978,9 @@ public class Balancer {
     // At last, match all remaining nodes
     chooseNodes(ANY_OTHER);
     
-    assert (datanodes.size() >= sources.size()+targets.size())
+    assert (datanodeMap.size() >= sources.size()+targets.size())
       : "Mismatched number of datanodes (" +
-      datanodes.size() + " total, " +
+      datanodeMap.size() + " total, " +
       sources.size() + " sources, " +
       targets.size() + " targets)";
 
@@ -1304,7 +1281,7 @@ public class Balancer {
     this.aboveAvgUtilizedDatanodes.clear();
     this.belowAvgUtilizedDatanodes.clear();
     this.underUtilizedDatanodes.clear();
-    this.datanodes.clear();
+    this.datanodeMap.clear();
     this.sources.clear();
     this.targets.clear();  
     this.policy.reset();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java

@@ -75,7 +75,7 @@ public interface BlockCollection {
    * and set the locations.
    */
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
-      DatanodeDescriptor[] locations) throws IOException;
+      DatanodeStorageInfo[] targets) throws IOException;
 
   /**
    * @return whether the block collection is under construction.

+ 91 - 41
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -21,6 +21,7 @@ import java.util.LinkedList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.util.LightWeightGSet;
 
@@ -39,11 +40,11 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   private LightWeightGSet.LinkedElement nextLinkedElement;
 
   /**
-   * This array contains triplets of references. For each i-th datanode the
-   * block belongs to triplets[3*i] is the reference to the DatanodeDescriptor
-   * and triplets[3*i+1] and triplets[3*i+2] are references to the previous and
-   * the next blocks, respectively, in the list of blocks belonging to this
-   * data-node.
+   * This array contains triplets of references. For each i-th storage, the
+   * block belongs to triplets[3*i] is the reference to the
+   * {@link DatanodeStorageInfo} and triplets[3*i+1] and triplets[3*i+2] are
+   * references to the previous and the next blocks, respectively, in the list
+   * of blocks belonging to this storage.
    * 
    * Using previous and next in Object triplets is done instead of a
    * {@link LinkedList} list to efficiently use memory. With LinkedList the cost
@@ -86,9 +87,14 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   }
 
   public DatanodeDescriptor getDatanode(int index) {
+    DatanodeStorageInfo storage = getStorageInfo(index);
+    return storage == null ? null : storage.getDatanodeDescriptor();
+  }
+
+  DatanodeStorageInfo getStorageInfo(int index) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-    return (DatanodeDescriptor)triplets[index*3];
+    return (DatanodeStorageInfo)triplets[index*3];
   }
 
   private BlockInfo getPrevious(int index) {
@@ -111,14 +117,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     return info;
   }
 
-  private void setDatanode(int index, DatanodeDescriptor node, BlockInfo previous,
-      BlockInfo next) {
+  private void setStorageInfo(int index, DatanodeStorageInfo storage) {
     assert this.triplets != null : "BlockInfo is not initialized";
-    int i = index * 3;
-    assert index >= 0 && i+2 < triplets.length : "Index is out of bound";
-    triplets[i] = node;
-    triplets[i+1] = previous;
-    triplets[i+2] = next;
+    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+    triplets[index*3] = storage;
   }
 
   /**
@@ -190,22 +192,34 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   }
 
   /**
-   * Add data-node this block belongs to.
+   * Add a {@link DatanodeStorageInfo} location for a block
    */
-  public boolean addNode(DatanodeDescriptor node) {
-    if(findDatanode(node) >= 0) // the node is already there
-      return false;
+  boolean addStorage(DatanodeStorageInfo storage) {
+    boolean added = true;
+    int idx = findDatanode(storage.getDatanodeDescriptor());
+    if(idx >= 0) {
+      if (getStorageInfo(idx) == storage) { // the storage is already there
+        return false;
+      } else {
+        // The block is on the DN but belongs to a different storage.
+        // Update our state.
+        removeStorage(storage);
+        added = false;      // Just updating storage. Return false.
+      }
+    }
     // find the last null node
     int lastNode = ensureCapacity(1);
-    setDatanode(lastNode, node, null, null);
-    return true;
+    setStorageInfo(lastNode, storage);
+    setNext(lastNode, null);
+    setPrevious(lastNode, null);
+    return added;
   }
 
   /**
-   * Remove data-node from the block.
+   * Remove {@link DatanodeStorageInfo} location for a block
    */
-  public boolean removeNode(DatanodeDescriptor node) {
-    int dnIndex = findDatanode(node);
+  boolean removeStorage(DatanodeStorageInfo storage) {
+    int dnIndex = findStorageInfo(storage);
     if(dnIndex < 0) // the node is not found
       return false;
     assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
@@ -213,10 +227,13 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     // find the last not null node
     int lastNode = numNodes()-1; 
     // replace current node triplet by the lastNode one 
-    setDatanode(dnIndex, getDatanode(lastNode), getPrevious(lastNode),
-        getNext(lastNode));
+    setStorageInfo(dnIndex, getStorageInfo(lastNode));
+    setNext(dnIndex, getNext(lastNode)); 
+    setPrevious(dnIndex, getPrevious(lastNode)); 
     // set the last triplet to null
-    setDatanode(lastNode, null, null, null);
+    setStorageInfo(lastNode, null);
+    setNext(lastNode, null); 
+    setPrevious(lastNode, null); 
     return true;
   }
 
@@ -236,37 +253,70 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     }
     return -1;
   }
+  /**
+   * Find specified DatanodeStorageInfo.
+   * @param dn
+   * @return index or -1 if not found.
+   */
+  int findStorageInfo(DatanodeInfo dn) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeStorageInfo cur = getStorageInfo(idx);
+      if(cur == null)
+        break;
+      if(cur.getDatanodeDescriptor() == dn)
+        return idx;
+    }
+    return -1;
+  }
+  
+  /**
+   * Find specified DatanodeStorageInfo.
+   * @param storageInfo
+   * @return index or -1 if not found.
+   */
+  int findStorageInfo(DatanodeStorageInfo storageInfo) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeStorageInfo cur = getStorageInfo(idx);
+      if(cur == storageInfo)
+        return idx;
+      if(cur == null)
+        break;
+    }
+    return -1;
+  }
 
   /**
    * Insert this block into the head of the list of blocks 
-   * related to the specified DatanodeDescriptor.
+   * related to the specified DatanodeStorageInfo.
    * If the head is null then form a new list.
    * @return current block as the new head of the list.
    */
-  public BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
-    int dnIndex = this.findDatanode(dn);
+  BlockInfo listInsert(BlockInfo head, DatanodeStorageInfo storage) {
+    int dnIndex = this.findStorageInfo(storage);
     assert dnIndex >= 0 : "Data node is not found: current";
     assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
             "Block is already in the list and cannot be inserted.";
     this.setPrevious(dnIndex, null);
     this.setNext(dnIndex, head);
     if(head != null)
-      head.setPrevious(head.findDatanode(dn), this);
+      head.setPrevious(head.findStorageInfo(storage), this);
     return this;
   }
 
   /**
    * Remove this block from the list of blocks 
-   * related to the specified DatanodeDescriptor.
+   * related to the specified DatanodeStorageInfo.
    * If this block is the head of the list then return the next block as 
    * the new head.
    * @return the new head of the list or null if the list becomes
-   * empty after deletion.
+   * empy after deletion.
    */
-  public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
+  BlockInfo listRemove(BlockInfo head, DatanodeStorageInfo storage) {
     if(head == null)
       return null;
-    int dnIndex = this.findDatanode(dn);
+    int dnIndex = this.findStorageInfo(storage);
     if(dnIndex < 0) // this block is not on the data-node list
       return head;
 
@@ -275,9 +325,9 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     this.setNext(dnIndex, null);
     this.setPrevious(dnIndex, null);
     if(prev != null)
-      prev.setNext(prev.findDatanode(dn), next);
+      prev.setNext(prev.findStorageInfo(storage), next);
     if(next != null)
-      next.setPrevious(next.findDatanode(dn), prev);
+      next.setPrevious(next.findStorageInfo(storage), prev);
     if(this == head)  // removing the head
       head = next;
     return head;
@@ -289,7 +339,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    *
    * @return the new head of the list.
    */
-  public BlockInfo moveBlockToHead(BlockInfo head, DatanodeDescriptor dn,
+  public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage,
       int curIndex, int headIndex) {
     if (head == this) {
       return this;
@@ -298,9 +348,9 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     BlockInfo prev = this.setPrevious(curIndex, null);
 
     head.setPrevious(headIndex, this);
-    prev.setNext(prev.findDatanode(dn), next);
+    prev.setNext(prev.findStorageInfo(storage), next);
     if (next != null)
-      next.setPrevious(next.findDatanode(dn), prev);
+      next.setPrevious(next.findStorageInfo(storage), prev);
     return this;
   }
 
@@ -328,10 +378,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * @return BlockInfoUnderConstruction -  an under construction block.
    */
   public BlockInfoUnderConstruction convertToBlockUnderConstruction(
-      BlockUCState s, DatanodeDescriptor[] targets) {
+      BlockUCState s, DatanodeStorageInfo[] targets) {
     if(isComplete()) {
-      return new BlockInfoUnderConstruction(
-          this, getBlockCollection().getBlockReplication(), s, targets);
+      return new BlockInfoUnderConstruction(this,
+          getBlockCollection().getBlockReplication(), s, targets);
     }
     // the block is already under construction
     BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;

+ 33 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -63,12 +63,12 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    * corresponding replicas.
    */
   static class ReplicaUnderConstruction extends Block {
-    private DatanodeDescriptor expectedLocation;
+    private final DatanodeStorageInfo expectedLocation;
     private ReplicaState state;
     private boolean chosenAsPrimary;
 
     ReplicaUnderConstruction(Block block,
-                             DatanodeDescriptor target,
+                             DatanodeStorageInfo target,
                              ReplicaState state) {
       super(block);
       this.expectedLocation = target;
@@ -82,7 +82,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
      * It is not guaranteed, but expected, that the data-node actually has
      * the replica.
      */
-    DatanodeDescriptor getExpectedLocation() {
+    private DatanodeStorageInfo getExpectedStorageLocation() {
       return expectedLocation;
     }
 
@@ -118,7 +118,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
      * Is data-node the replica belongs to alive.
      */
     boolean isAlive() {
-      return expectedLocation.isAlive;
+      return expectedLocation.getDatanodeDescriptor().isAlive;
     }
 
     @Override // Block
@@ -162,7 +162,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    */
   public BlockInfoUnderConstruction(Block blk, int replication,
                              BlockUCState state,
-                             DatanodeDescriptor[] targets) {
+                             DatanodeStorageInfo[] targets) {
     super(blk, replication);
     assert getBlockUCState() != BlockUCState.COMPLETE :
       "BlockInfoUnderConstruction cannot be in COMPLETE state";
@@ -186,7 +186,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
   }
 
   /** Set expected locations */
-  public void setExpectedLocations(DatanodeDescriptor[] targets) {
+  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
     for(int i = 0; i < numLocations; i++)
@@ -198,12 +198,12 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    * Create array of expected replica locations
    * (as has been assigned by chooseTargets()).
    */
-  public DatanodeDescriptor[] getExpectedLocations() {
+  public DatanodeStorageInfo[] getExpectedStorageLocations() {
     int numLocations = replicas == null ? 0 : replicas.size();
-    DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations];
+    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
     for(int i = 0; i < numLocations; i++)
-      locations[i] = replicas.get(i).getExpectedLocation();
-    return locations;
+      storages[i] = replicas.get(i).getExpectedStorageLocation();
+    return storages;
   }
 
   /** Get the number of expected locations */
@@ -244,9 +244,9 @@ public class BlockInfoUnderConstruction extends BlockInfo {
     // The replica list is unchanged.
     for (ReplicaUnderConstruction r : replicas) {
       if (genStamp != r.getGenerationStamp()) {
-        r.getExpectedLocation().removeBlock(this);
+        r.getExpectedStorageLocation().removeBlock(this);
         NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
-            + "from location: " + r.getExpectedLocation());
+            + "from location: " + r.getExpectedStorageLocation());
       }
     }
   }
@@ -302,31 +302,44 @@ public class BlockInfoUnderConstruction extends BlockInfo {
       if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
         continue;
       }
-      if (replicas.get(i).getExpectedLocation().getLastUpdate() > mostRecentLastUpdate) {
-        primary = replicas.get(i);
+      final ReplicaUnderConstruction ruc = replicas.get(i);
+      final long lastUpdate = ruc.getExpectedStorageLocation().getDatanodeDescriptor().getLastUpdate(); 
+      if (lastUpdate > mostRecentLastUpdate) {
         primaryNodeIndex = i;
-        mostRecentLastUpdate = primary.getExpectedLocation().getLastUpdate();
+        primary = ruc;
+        mostRecentLastUpdate = lastUpdate;
       }
     }
     if (primary != null) {
-      primary.getExpectedLocation().addBlockToBeRecovered(this);
+      primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
       primary.setChosenAsPrimary(true);
       NameNode.blockStateChangeLog.info("BLOCK* " + this
         + " recovery started, primary=" + primary);
     }
   }
 
-  void addReplicaIfNotPresent(DatanodeDescriptor dn,
+  void addReplicaIfNotPresent(DatanodeStorageInfo storage,
                      Block block,
                      ReplicaState rState) {
-    for (ReplicaUnderConstruction r : replicas) {
-      if (r.getExpectedLocation() == dn) {
+    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
+    while (it.hasNext()) {
+      ReplicaUnderConstruction r = it.next();
+      if(r.getExpectedStorageLocation() == storage) {
         // Record the gen stamp from the report
         r.setGenerationStamp(block.getGenerationStamp());
         return;
+      } else if (r.getExpectedStorageLocation().getDatanodeDescriptor() ==
+          storage.getDatanodeDescriptor()) {
+
+        // The Datanode reported that the block is on a different storage
+        // than the one chosen by BlockPlacementPolicy. This can occur as
+        // we allow Datanodes to choose the target storage. Update our
+        // state by removing the stale entry and adding a new one.
+        it.remove();
+        break;
       }
     }
-    replicas.add(new ReplicaUnderConstruction(block, dn, rState));
+    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
   }
 
   @Override // BlockInfo

文件差異過大導致無法顯示
+ 193 - 165
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java


+ 12 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -67,13 +68,14 @@ public abstract class BlockPlacementPolicy {
    * @return array of DatanodeDescriptor instances chosen as target
    * and sorted as a pipeline.
    */
-  public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
+  public abstract DatanodeStorageInfo[] chooseTarget(String srcPath,
                                              int numOfReplicas,
                                              Node writer,
-                                             List<DatanodeDescriptor> chosenNodes,
+                                             List<DatanodeStorageInfo> chosen,
                                              boolean returnChosenNodes,
                                              Set<Node> excludedNodes,
-                                             long blocksize);
+                                             long blocksize,
+                                             StorageType storageType);
   
   /**
    * Same as {@link #chooseTarget(String, int, Node, List, boolean, 
@@ -82,16 +84,19 @@ public abstract class BlockPlacementPolicy {
    *          is only a hint and due to cluster state, namenode may not be 
    *          able to place the blocks on these datanodes.
    */
-  DatanodeDescriptor[] chooseTarget(String src,
+  DatanodeStorageInfo[] chooseTarget(String src,
       int numOfReplicas, Node writer,
       Set<Node> excludedNodes,
-      long blocksize, List<DatanodeDescriptor> favoredNodes) {
+      long blocksize,
+      List<DatanodeDescriptor> favoredNodes,
+      StorageType storageType) {
     // This class does not provide the functionality of placing
     // a block in favored datanodes. The implementations of this class
     // are expected to provide this functionality
+
     return chooseTarget(src, numOfReplicas, writer, 
-        new ArrayList<DatanodeDescriptor>(numOfReplicas), false, excludedNodes, 
-        blocksize);
+        new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
+        excludedNodes, blocksize, storageType);
   }
 
   /**

+ 186 - 139
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -29,11 +29,14 @@ import java.util.TreeSet;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
@@ -103,99 +106,101 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
 
   @Override
-  public DatanodeDescriptor[] chooseTarget(String srcPath,
+  public DatanodeStorageInfo[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     Node writer,
-                                    List<DatanodeDescriptor> chosenNodes,
+                                    List<DatanodeStorageInfo> chosenNodes,
                                     boolean returnChosenNodes,
                                     Set<Node> excludedNodes,
-                                    long blocksize) {
+                                    long blocksize,
+                                    StorageType storageType) {
     return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
-        excludedNodes, blocksize);
+        excludedNodes, blocksize, storageType);
   }
 
   @Override
-  DatanodeDescriptor[] chooseTarget(String src,
+  DatanodeStorageInfo[] chooseTarget(String src,
       int numOfReplicas,
       Node writer,
       Set<Node> excludedNodes,
       long blocksize,
-      List<DatanodeDescriptor> favoredNodes) {
+      List<DatanodeDescriptor> favoredNodes,
+      StorageType storageType) {
     try {
       if (favoredNodes == null || favoredNodes.size() == 0) {
         // Favored nodes not specified, fall back to regular block placement.
         return chooseTarget(src, numOfReplicas, writer,
-            new ArrayList<DatanodeDescriptor>(numOfReplicas), false, 
-            excludedNodes, blocksize);
+            new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, 
+            excludedNodes, blocksize, storageType);
       }
 
       Set<Node> favoriteAndExcludedNodes = excludedNodes == null ?
           new HashSet<Node>() : new HashSet<Node>(excludedNodes);
 
       // Choose favored nodes
-      List<DatanodeDescriptor> results = new ArrayList<DatanodeDescriptor>();
+      List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>();
       boolean avoidStaleNodes = stats != null
           && stats.isAvoidingStaleDataNodesForWrite();
       for (int i = 0; i < Math.min(favoredNodes.size(), numOfReplicas); i++) {
         DatanodeDescriptor favoredNode = favoredNodes.get(i);
         // Choose a single node which is local to favoredNode.
         // 'results' is updated within chooseLocalNode
-        DatanodeDescriptor target = chooseLocalNode(favoredNode,
+        final DatanodeStorageInfo target = chooseLocalStorage(favoredNode,
             favoriteAndExcludedNodes, blocksize, 
-            getMaxNodesPerRack(results, 
-                numOfReplicas)[1], results, avoidStaleNodes);
+            getMaxNodesPerRack(results.size(), numOfReplicas)[1],
+            results, avoidStaleNodes, storageType);
         if (target == null) {
           LOG.warn("Could not find a target for file " + src
               + " with favored node " + favoredNode); 
           continue;
         }
-        favoriteAndExcludedNodes.add(target);
+        favoriteAndExcludedNodes.add(target.getDatanodeDescriptor());
       }
 
       if (results.size() < numOfReplicas) {
         // Not enough favored nodes, choose other nodes.
         numOfReplicas -= results.size();
-        DatanodeDescriptor[] remainingTargets = 
+        DatanodeStorageInfo[] remainingTargets = 
             chooseTarget(src, numOfReplicas, writer, results,
-                false, favoriteAndExcludedNodes, blocksize);
+                false, favoriteAndExcludedNodes, blocksize, storageType);
         for (int i = 0; i < remainingTargets.length; i++) {
           results.add(remainingTargets[i]);
         }
       }
       return getPipeline(writer,
-          results.toArray(new DatanodeDescriptor[results.size()]));
+          results.toArray(new DatanodeStorageInfo[results.size()]));
     } catch (NotEnoughReplicasException nr) {
       // Fall back to regular block placement disregarding favored nodes hint
       return chooseTarget(src, numOfReplicas, writer, 
-          new ArrayList<DatanodeDescriptor>(numOfReplicas), false, 
-          excludedNodes, blocksize);
+          new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, 
+          excludedNodes, blocksize, storageType);
     }
   }
 
   /** This is the implementation. */
-  private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+  private DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
                                     Node writer,
-                                    List<DatanodeDescriptor> chosenNodes,
+                                    List<DatanodeStorageInfo> chosenStorage,
                                     boolean returnChosenNodes,
                                     Set<Node> excludedNodes,
-                                    long blocksize) {
+                                    long blocksize,
+                                    StorageType storageType) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
-      return DatanodeDescriptor.EMPTY_ARRAY;
+      return DatanodeStorageInfo.EMPTY_ARRAY;
     }
       
     if (excludedNodes == null) {
       excludedNodes = new HashSet<Node>();
     }
      
-    int[] result = getMaxNodesPerRack(chosenNodes, numOfReplicas);
+    int[] result = getMaxNodesPerRack(chosenStorage.size(), numOfReplicas);
     numOfReplicas = result[0];
     int maxNodesPerRack = result[1];
       
-    List<DatanodeDescriptor> results = 
-      new ArrayList<DatanodeDescriptor>(chosenNodes);
-    for (DatanodeDescriptor node:chosenNodes) {
+    final List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>(chosenStorage);
+    for (DatanodeStorageInfo storage : chosenStorage) {
       // add localMachine and related nodes to excludedNodes
-      addToExcludedNodes(node, excludedNodes);
+      addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes);
     }
       
     if (!clusterMap.contains(writer)) {
@@ -205,20 +210,19 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     boolean avoidStaleNodes = (stats != null
         && stats.isAvoidingStaleDataNodesForWrite());
     Node localNode = chooseTarget(numOfReplicas, writer,
-        excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
+        excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
     if (!returnChosenNodes) {  
-      results.removeAll(chosenNodes);
+      results.removeAll(chosenStorage);
     }
       
     // sorting nodes to form a pipeline
     return getPipeline((writer==null)?localNode:writer,
-                       results.toArray(new DatanodeDescriptor[results.size()]));
+                       results.toArray(new DatanodeStorageInfo[results.size()]));
   }
 
-  private int[] getMaxNodesPerRack(List<DatanodeDescriptor> chosenNodes,
-      int numOfReplicas) {
+  private int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) {
     int clusterSize = clusterMap.getNumOfLeaves();
-    int totalNumOfReplicas = chosenNodes.size()+numOfReplicas;
+    int totalNumOfReplicas = numOfChosen + numOfReplicas;
     if (totalNumOfReplicas > clusterSize) {
       numOfReplicas -= (totalNumOfReplicas-clusterSize);
       totalNumOfReplicas = clusterSize;
@@ -243,8 +247,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                           Set<Node> excludedNodes,
                                           long blocksize,
                                           int maxNodesPerRack,
-                                          List<DatanodeDescriptor> results,
-                                          final boolean avoidStaleNodes) {
+                                          List<DatanodeStorageInfo> results,
+                                          final boolean avoidStaleNodes,
+                                          StorageType storageType) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
       return writer;
     }
@@ -253,7 +258,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     int numOfResults = results.size();
     boolean newBlock = (numOfResults==0);
     if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) {
-      writer = results.get(0);
+      writer = results.get(0).getDatanodeDescriptor();
     }
 
     // Keep a copy of original excludedNodes
@@ -261,42 +266,49 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         new HashSet<Node>(excludedNodes) : null;
     try {
       if (numOfResults == 0) {
-        writer = chooseLocalNode(writer, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+        writer = chooseLocalStorage(writer, excludedNodes, blocksize,
+            maxNodesPerRack, results, avoidStaleNodes, storageType)
+                .getDatanodeDescriptor();
         if (--numOfReplicas == 0) {
           return writer;
         }
       }
+      final DatanodeDescriptor dn0 = results.get(0).getDatanodeDescriptor();
       if (numOfResults <= 1) {
-        chooseRemoteRack(1, results.get(0), excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+        chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
+            results, avoidStaleNodes, storageType);
         if (--numOfReplicas == 0) {
           return writer;
         }
       }
       if (numOfResults <= 2) {
-        if (clusterMap.isOnSameRack(results.get(0), results.get(1))) {
-          chooseRemoteRack(1, results.get(0), excludedNodes,
-                           blocksize, maxNodesPerRack, 
-                           results, avoidStaleNodes);
+        final DatanodeDescriptor dn1 = results.get(1).getDatanodeDescriptor();
+        if (clusterMap.isOnSameRack(dn0, dn1)) {
+          chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
+              results, avoidStaleNodes, storageType);
         } else if (newBlock){
-          chooseLocalRack(results.get(1), excludedNodes, blocksize, 
-                          maxNodesPerRack, results, avoidStaleNodes);
+          chooseLocalRack(dn1, excludedNodes, blocksize, maxNodesPerRack,
+              results, avoidStaleNodes, storageType);
         } else {
           chooseLocalRack(writer, excludedNodes, blocksize, maxNodesPerRack,
-              results, avoidStaleNodes);
+              results, avoidStaleNodes, storageType);
         }
         if (--numOfReplicas == 0) {
           return writer;
         }
       }
       chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes);
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e) {
-      LOG.warn("Not able to place enough replicas, still in need of "
-               + (totalReplicasExpected - results.size()) + " to reach "
-               + totalReplicasExpected + "\n"
-               + e.getMessage());
+      final String message = "Failed to place enough replicas, still in need of "
+          + (totalReplicasExpected - results.size()) + " to reach "
+          + totalReplicasExpected + ".";
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(message, e);
+      } else {
+        LOG.warn(message + " " + e.getMessage());
+      }
+
       if (avoidStaleNodes) {
         // Retry chooseTarget again, this time not avoiding stale nodes.
 
@@ -304,14 +316,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         // not chosen because they were stale, decommissioned, etc.
         // We need to additionally exclude the nodes that were added to the 
         // result list in the successful calls to choose*() above.
-        for (Node node : results) {
-          oldExcludedNodes.add(node);
+        for (DatanodeStorageInfo resultStorage : results) {
+          oldExcludedNodes.add(resultStorage.getDatanodeDescriptor());
         }
         // Set numOfReplicas, since it can get out of sync with the result list
         // if the NotEnoughReplicasException was thrown in chooseRandom().
         numOfReplicas = totalReplicasExpected - results.size();
         return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
-            maxNodesPerRack, results, false);
+            maxNodesPerRack, results, false, storageType);
       }
     }
     return writer;
@@ -321,32 +333,36 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * Choose <i>localMachine</i> as the target.
    * if <i>localMachine</i> is not available, 
    * choose a node on the same rack
-   * @return the chosen node
+   * @return the chosen storage
    */
-  protected DatanodeDescriptor chooseLocalNode(Node localMachine,
+  protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
                                              Set<Node> excludedNodes,
                                              long blocksize,
                                              int maxNodesPerRack,
-                                             List<DatanodeDescriptor> results,
-                                             boolean avoidStaleNodes)
+                                             List<DatanodeStorageInfo> results,
+                                             boolean avoidStaleNodes,
+                                             StorageType storageType)
       throws NotEnoughReplicasException {
     // if no local machine, randomly choose one node
     if (localMachine == null)
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes);
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
       DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
       // otherwise try local machine first
       if (excludedNodes.add(localMachine)) { // was not in the excluded list
-        if (addIfIsGoodTarget(localDatanode, excludedNodes, blocksize,
-            maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
-          return localDatanode;
+        for(DatanodeStorageInfo localStorage : DFSUtil.shuffle(
+            localDatanode.getStorageInfos())) {
+          if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
+              maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) {
+            return localStorage;
+          }
         }
       } 
     }      
     // try a node on local rack
     return chooseLocalRack(localMachine, excludedNodes, blocksize,
-        maxNodesPerRack, results, avoidStaleNodes);
+        maxNodesPerRack, results, avoidStaleNodes, storageType);
   }
   
   /**
@@ -368,27 +384,29 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * in the cluster.
    * @return the chosen node
    */
-  protected DatanodeDescriptor chooseLocalRack(Node localMachine,
+  protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
                                              Set<Node> excludedNodes,
                                              long blocksize,
                                              int maxNodesPerRack,
-                                             List<DatanodeDescriptor> results,
-                                             boolean avoidStaleNodes)
+                                             List<DatanodeStorageInfo> results,
+                                             boolean avoidStaleNodes,
+                                             StorageType storageType)
       throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     if (localMachine == null) {
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes);
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     }
       
     // choose one from the local rack
     try {
       return chooseRandom(localMachine.getNetworkLocation(), excludedNodes,
-          blocksize, maxNodesPerRack, results, avoidStaleNodes);
+          blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e1) {
       // find the second replica
       DatanodeDescriptor newLocal=null;
-      for(DatanodeDescriptor nextNode : results) {
+      for(DatanodeStorageInfo resultStorage : results) {
+        DatanodeDescriptor nextNode = resultStorage.getDatanodeDescriptor();
         if (nextNode != localMachine) {
           newLocal = nextNode;
           break;
@@ -397,16 +415,16 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       if (newLocal != null) {
         try {
           return chooseRandom(newLocal.getNetworkLocation(), excludedNodes,
-              blocksize, maxNodesPerRack, results, avoidStaleNodes);
+              blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
         } catch(NotEnoughReplicasException e2) {
           //otherwise randomly choose one from the network
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-              maxNodesPerRack, results, avoidStaleNodes);
+              maxNodesPerRack, results, avoidStaleNodes, storageType);
         }
       } else {
         //otherwise randomly choose one from the network
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+            maxNodesPerRack, results, avoidStaleNodes, storageType);
       }
     }
   }
@@ -423,48 +441,51 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                 Set<Node> excludedNodes,
                                 long blocksize,
                                 int maxReplicasPerRack,
-                                List<DatanodeDescriptor> results,
-                                boolean avoidStaleNodes)
+                                List<DatanodeStorageInfo> results,
+                                boolean avoidStaleNodes,
+                                StorageType storageType)
                                     throws NotEnoughReplicasException {
     int oldNumOfReplicas = results.size();
     // randomly choose one node from remote racks
     try {
       chooseRandom(numOfReplicas, "~" + localMachine.getNetworkLocation(),
           excludedNodes, blocksize, maxReplicasPerRack, results,
-          avoidStaleNodes);
+          avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e) {
       chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
                    localMachine.getNetworkLocation(), excludedNodes, blocksize, 
-                   maxReplicasPerRack, results, avoidStaleNodes);
+                   maxReplicasPerRack, results, avoidStaleNodes, storageType);
     }
   }
 
   /**
    * Randomly choose one target from the given <i>scope</i>.
-   * @return the chosen node, if there is any.
+   * @return the chosen storage, if there is any.
    */
-  protected DatanodeDescriptor chooseRandom(String scope,
+  protected DatanodeStorageInfo chooseRandom(String scope,
       Set<Node> excludedNodes,
       long blocksize,
       int maxNodesPerRack,
-      List<DatanodeDescriptor> results,
-      boolean avoidStaleNodes)
+      List<DatanodeStorageInfo> results,
+      boolean avoidStaleNodes,
+      StorageType storageType)
           throws NotEnoughReplicasException {
     return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack,
-        results, avoidStaleNodes);
+        results, avoidStaleNodes, storageType);
   }
 
   /**
    * Randomly choose <i>numOfReplicas</i> targets from the given <i>scope</i>.
    * @return the first chosen node, if there is any.
    */
-  protected DatanodeDescriptor chooseRandom(int numOfReplicas,
+  protected DatanodeStorageInfo chooseRandom(int numOfReplicas,
                             String scope,
                             Set<Node> excludedNodes,
                             long blocksize,
                             int maxNodesPerRack,
-                            List<DatanodeDescriptor> results,
-                            boolean avoidStaleNodes)
+                            List<DatanodeStorageInfo> results,
+                            boolean avoidStaleNodes,
+                            StorageType storageType)
                                 throws NotEnoughReplicasException {
       
     int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
@@ -476,24 +497,32 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       builder.append("[");
     }
     boolean badTarget = false;
-    DatanodeDescriptor firstChosen = null;
+    DatanodeStorageInfo firstChosen = null;
     while(numOfReplicas > 0 && numOfAvailableNodes > 0) {
       DatanodeDescriptor chosenNode = 
           (DatanodeDescriptor)clusterMap.chooseRandom(scope);
       if (excludedNodes.add(chosenNode)) { //was not in the excluded list
         numOfAvailableNodes--;
 
-        int newExcludedNodes = addIfIsGoodTarget(chosenNode, excludedNodes,
-            blocksize, maxNodesPerRack, considerLoad, results, avoidStaleNodes);
-        if (newExcludedNodes >= 0) {
-          numOfReplicas--;
-          if (firstChosen == null) {
-            firstChosen = chosenNode;
+        final DatanodeStorageInfo[] storages = DFSUtil.shuffle(
+            chosenNode.getStorageInfos());
+        int i;
+        for(i = 0; i < storages.length; i++) {
+          final int newExcludedNodes = addIfIsGoodTarget(storages[i],
+              excludedNodes, blocksize, maxNodesPerRack, considerLoad, results,
+              avoidStaleNodes, storageType);
+          if (newExcludedNodes >= 0) {
+            numOfReplicas--;
+            if (firstChosen == null) {
+              firstChosen = storages[i];
+            }
+            numOfAvailableNodes -= newExcludedNodes;
+            break;
           }
-          numOfAvailableNodes -= newExcludedNodes;
-        } else {
-          badTarget = true;
         }
+
+        // If no candidate storage was found on this DN then set badTarget.
+        badTarget = (i == storages.length);
       }
     }
       
@@ -512,43 +541,46 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
 
   /**
-   * If the given node is a good target, add it to the result list and
+   * If the given storage is a good target, add it to the result list and
    * update the set of excluded nodes.
    * @return -1 if the given is not a good target;
    *         otherwise, return the number of nodes added to excludedNodes set.
    */
-  int addIfIsGoodTarget(DatanodeDescriptor node,
+  int addIfIsGoodTarget(DatanodeStorageInfo storage,
       Set<Node> excludedNodes,
       long blockSize,
       int maxNodesPerRack,
       boolean considerLoad,
-      List<DatanodeDescriptor> results,
-      boolean avoidStaleNodes) {
-    if (isGoodTarget(node, blockSize, maxNodesPerRack, considerLoad,
-        results, avoidStaleNodes)) {
-      results.add(node);
+      List<DatanodeStorageInfo> results,                           
+      boolean avoidStaleNodes,
+      StorageType storageType) {
+    if (isGoodTarget(storage, blockSize, maxNodesPerRack, considerLoad,
+        results, avoidStaleNodes, storageType)) {
+      results.add(storage);
       // add node and related nodes to excludedNode
-      return addToExcludedNodes(node, excludedNodes);
+      return addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes);
     } else { 
       return -1;
     }
   }
 
-  private static void logNodeIsNotChosen(DatanodeDescriptor node, String reason) {
+  private static void logNodeIsNotChosen(DatanodeStorageInfo storage, String reason) {
     if (LOG.isDebugEnabled()) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       // build the error message for later use.
       debugLoggingBuilder.get()
           .append(node).append(": ")
-          .append("Node ").append(NodeBase.getPath(node))
+          .append("Storage ").append(storage)
+          .append("at node ").append(NodeBase.getPath(node))
           .append(" is not chosen because ")
           .append(reason);
     }
   }
 
   /**
-   * Determine if a node is a good target. 
+   * Determine if a storage is a good target. 
    * 
-   * @param node The target node
+   * @param storage The target storage
    * @param blockSize Size of block
    * @param maxTargetPerRack Maximum number of targets per rack. The value of 
    *                       this parameter depends on the number of racks in 
@@ -561,32 +593,43 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    *         does not have too much load, 
    *         and the rack does not have too many nodes.
    */
-  private boolean isGoodTarget(DatanodeDescriptor node,
+  private boolean isGoodTarget(DatanodeStorageInfo storage,
                                long blockSize, int maxTargetPerRack,
                                boolean considerLoad,
-                               List<DatanodeDescriptor> results,                           
-                               boolean avoidStaleNodes) {
-    // check if the node is (being) decommissed
+                               List<DatanodeStorageInfo> results,
+                               boolean avoidStaleNodes,
+                               StorageType storageType) {
+    if (storage.getStorageType() != storageType) {
+      logNodeIsNotChosen(storage,
+          "storage types do not match, where the expected storage type is "
+              + storageType);
+      return false;
+    }
+    if (storage.getState() == State.READ_ONLY) {
+      logNodeIsNotChosen(storage, "storage is read-only");
+      return false;
+    }
+    DatanodeDescriptor node = storage.getDatanodeDescriptor();
+    // check if the node is (being) decommissioned
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
-      logNodeIsNotChosen(node, "the node is (being) decommissioned ");
+      logNodeIsNotChosen(storage, "the node is (being) decommissioned ");
       return false;
     }
 
     if (avoidStaleNodes) {
       if (node.isStale(this.staleInterval)) {
-        logNodeIsNotChosen(node, "the node is stale ");
+        logNodeIsNotChosen(storage, "the node is stale ");
         return false;
       }
     }
     
-    long remaining = node.getRemaining() - 
-                     (node.getBlocksScheduled() * blockSize); 
-    // check the remaining capacity of the target machine
-    if (blockSize* HdfsConstants.MIN_BLOCKS_FOR_WRITE>remaining) {
-      logNodeIsNotChosen(node, "the node does not have enough space ");
+    final long requiredSize = blockSize * HdfsConstants.MIN_BLOCKS_FOR_WRITE;
+    final long scheduledSize = blockSize * node.getBlocksScheduled();
+    if (requiredSize > node.getRemaining() - scheduledSize) {
+      logNodeIsNotChosen(storage, "the node does not have enough space ");
       return false;
     }
-      
+
     // check the communication traffic of the target machine
     if (considerLoad) {
       double avgLoad = 0;
@@ -595,7 +638,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         avgLoad = (double)stats.getTotalLoad()/size;
       }
       if (node.getXceiverCount() > (2.0 * avgLoad)) {
-        logNodeIsNotChosen(node, "the node is too busy ");
+        logNodeIsNotChosen(storage, "the node is too busy ");
         return false;
       }
     }
@@ -603,13 +646,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     // check if the target rack has chosen too many nodes
     String rackname = node.getNetworkLocation();
     int counter=1;
-    for(Node result : results) {
-      if (rackname.equals(result.getNetworkLocation())) {
+    for(DatanodeStorageInfo resultStorage : results) {
+      if (rackname.equals(
+          resultStorage.getDatanodeDescriptor().getNetworkLocation())) {
         counter++;
       }
     }
     if (counter>maxTargetPerRack) {
-      logNodeIsNotChosen(node, "the rack has too many chosen nodes ");
+      logNodeIsNotChosen(storage, "the rack has too many chosen nodes ");
       return false;
     }
     return true;
@@ -621,37 +665,40 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * starts from the writer and traverses all <i>nodes</i>
    * This is basically a traveling salesman problem.
    */
-  private DatanodeDescriptor[] getPipeline(Node writer,
-                                           DatanodeDescriptor[] nodes) {
-    if (nodes.length==0) return nodes;
-      
+  private DatanodeStorageInfo[] getPipeline(Node writer,
+      DatanodeStorageInfo[] storages) {
+    if (storages.length == 0) {
+      return storages;
+    }
+
     synchronized(clusterMap) {
       int index=0;
       if (writer == null || !clusterMap.contains(writer)) {
-        writer = nodes[0];
+        writer = storages[0].getDatanodeDescriptor();
       }
-      for(;index<nodes.length; index++) {
-        DatanodeDescriptor shortestNode = nodes[index];
-        int shortestDistance = clusterMap.getDistance(writer, shortestNode);
+      for(; index < storages.length; index++) {
+        DatanodeStorageInfo shortestStorage = storages[index];
+        int shortestDistance = clusterMap.getDistance(writer,
+            shortestStorage.getDatanodeDescriptor());
         int shortestIndex = index;
-        for(int i=index+1; i<nodes.length; i++) {
-          DatanodeDescriptor currentNode = nodes[i];
-          int currentDistance = clusterMap.getDistance(writer, currentNode);
+        for(int i = index + 1; i < storages.length; i++) {
+          int currentDistance = clusterMap.getDistance(writer,
+              storages[i].getDatanodeDescriptor());
           if (shortestDistance>currentDistance) {
             shortestDistance = currentDistance;
-            shortestNode = currentNode;
+            shortestStorage = storages[i];
             shortestIndex = i;
           }
         }
         //switch position index & shortestIndex
         if (index != shortestIndex) {
-          nodes[shortestIndex] = nodes[index];
-          nodes[index] = shortestNode;
+          storages[shortestIndex] = storages[index];
+          storages[index] = shortestStorage;
         }
-        writer = shortestNode;
+        writer = shortestStorage.getDatanodeDescriptor();
       }
     }
-    return nodes;
+    return storages;
   }
 
   @Override

+ 59 - 56
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java

@@ -25,6 +25,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
 import org.apache.hadoop.net.NetworkTopology;
@@ -64,81 +66,87 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
    * @return the chosen node
    */
   @Override
-  protected DatanodeDescriptor chooseLocalNode(Node localMachine,
+  protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
-      List<DatanodeDescriptor> results, boolean avoidStaleNodes)
-        throws NotEnoughReplicasException {
+      List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
+      StorageType storageType) throws NotEnoughReplicasException {
     // if no local machine, randomly choose one node
     if (localMachine == null)
       return chooseRandom(NodeBase.ROOT, excludedNodes, 
-          blocksize, maxNodesPerRack, results, avoidStaleNodes);
+          blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
 
+    // otherwise try local machine first
     if (localMachine instanceof DatanodeDescriptor) {
       DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine;
-      // otherwise try local machine first
       if (excludedNodes.add(localMachine)) { // was not in the excluded list
-        if (addIfIsGoodTarget(localDataNode, excludedNodes, blocksize,
-            maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
-          return localDataNode;
+        for(DatanodeStorageInfo localStorage : DFSUtil.shuffle(
+            localDataNode.getStorageInfos())) {
+          if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
+              maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) {
+            return localStorage;
+          }
         }
       }
     }
 
     // try a node on local node group
-    DatanodeDescriptor chosenNode = chooseLocalNodeGroup(
+    DatanodeStorageInfo chosenStorage = chooseLocalNodeGroup(
         (NetworkTopologyWithNodeGroup)clusterMap, localMachine, excludedNodes, 
-        blocksize, maxNodesPerRack, results, avoidStaleNodes);
-    if (chosenNode != null) {
-      return chosenNode;
+        blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
+    if (chosenStorage != null) {
+      return chosenStorage;
     }
     // try a node on local rack
     return chooseLocalRack(localMachine, excludedNodes, 
-        blocksize, maxNodesPerRack, results, avoidStaleNodes);
+        blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
+  }
+
+  /** @return the node of the second replica */
+  private static DatanodeDescriptor secondNode(Node localMachine,
+      List<DatanodeStorageInfo> results) {
+    // find the second replica
+    for(DatanodeStorageInfo nextStorage : results) {
+      DatanodeDescriptor nextNode = nextStorage.getDatanodeDescriptor();
+      if (nextNode != localMachine) {
+        return nextNode;
+      }
+    }
+    return null;
   }
 
-  
   @Override
-  protected DatanodeDescriptor chooseLocalRack(Node localMachine,
+  protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
-      List<DatanodeDescriptor> results, boolean avoidStaleNodes)
-      throws NotEnoughReplicasException {
+      List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
+      StorageType storageType) throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     if (localMachine == null) {
-      return chooseRandom(NodeBase.ROOT, excludedNodes, 
-                          blocksize, maxNodesPerRack, results, 
-                          avoidStaleNodes);
+      return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     }
 
     // choose one from the local rack, but off-nodegroup
     try {
-      return chooseRandom(NetworkTopology.getFirstHalf(
-                              localMachine.getNetworkLocation()),
-                          excludedNodes, blocksize, 
-                          maxNodesPerRack, results, 
-                          avoidStaleNodes);
+      final String scope = NetworkTopology.getFirstHalf(localMachine.getNetworkLocation());
+      return chooseRandom(scope, excludedNodes, blocksize, maxNodesPerRack,
+          results, avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e1) {
       // find the second replica
-      DatanodeDescriptor newLocal=null;
-      for(DatanodeDescriptor nextNode : results) {
-        if (nextNode != localMachine) {
-          newLocal = nextNode;
-          break;
-        }
-      }
+      final DatanodeDescriptor newLocal = secondNode(localMachine, results);
       if (newLocal != null) {
         try {
           return chooseRandom(
               clusterMap.getRack(newLocal.getNetworkLocation()), excludedNodes,
-              blocksize, maxNodesPerRack, results, avoidStaleNodes);
+              blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
         } catch(NotEnoughReplicasException e2) {
           //otherwise randomly choose one from the network
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-              maxNodesPerRack, results, avoidStaleNodes);
+              maxNodesPerRack, results, avoidStaleNodes, storageType);
         }
       } else {
         //otherwise randomly choose one from the network
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+            maxNodesPerRack, results, avoidStaleNodes, storageType);
       }
     }
   }
@@ -146,8 +154,9 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
   @Override
   protected void chooseRemoteRack(int numOfReplicas,
       DatanodeDescriptor localMachine, Set<Node> excludedNodes,
-      long blocksize, int maxReplicasPerRack, List<DatanodeDescriptor> results,
-      boolean avoidStaleNodes) throws NotEnoughReplicasException {
+      long blocksize, int maxReplicasPerRack, List<DatanodeStorageInfo> results,
+      boolean avoidStaleNodes, StorageType storageType)
+          throws NotEnoughReplicasException {
     int oldNumOfReplicas = results.size();
 
     final String rackLocation = NetworkTopology.getFirstHalf(
@@ -155,12 +164,12 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
     try {
       // randomly choose from remote racks
       chooseRandom(numOfReplicas, "~" + rackLocation, excludedNodes, blocksize,
-          maxReplicasPerRack, results, avoidStaleNodes);
+          maxReplicasPerRack, results, avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e) {
       // fall back to the local rack
       chooseRandom(numOfReplicas - (results.size() - oldNumOfReplicas),
           rackLocation, excludedNodes, blocksize,
-          maxReplicasPerRack, results, avoidStaleNodes);
+          maxReplicasPerRack, results, avoidStaleNodes, storageType);
     }
   }
 
@@ -170,46 +179,40 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
    * if still no such node is available, choose a random node in the cluster.
    * @return the chosen node
    */
-  private DatanodeDescriptor chooseLocalNodeGroup(
+  private DatanodeStorageInfo chooseLocalNodeGroup(
       NetworkTopologyWithNodeGroup clusterMap, Node localMachine,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
-      List<DatanodeDescriptor> results, boolean avoidStaleNodes)
-      throws NotEnoughReplicasException {
+      List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
+      StorageType storageType) throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     if (localMachine == null) {
-      return chooseRandom(NodeBase.ROOT, excludedNodes, 
-      blocksize, maxNodesPerRack, results, avoidStaleNodes);
+      return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     }
 
     // choose one from the local node group
     try {
       return chooseRandom(
           clusterMap.getNodeGroup(localMachine.getNetworkLocation()),
-          excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
+          excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes,
+          storageType);
     } catch (NotEnoughReplicasException e1) {
-      // find the second replica
-      DatanodeDescriptor newLocal=null;
-      for(DatanodeDescriptor nextNode : results) {
-        if (nextNode != localMachine) {
-          newLocal = nextNode;
-          break;
-        }
-      }
+      final DatanodeDescriptor newLocal = secondNode(localMachine, results);
       if (newLocal != null) {
         try {
           return chooseRandom(
               clusterMap.getNodeGroup(newLocal.getNetworkLocation()),
               excludedNodes, blocksize, maxNodesPerRack, results,
-              avoidStaleNodes);
+              avoidStaleNodes, storageType);
         } catch(NotEnoughReplicasException e2) {
           //otherwise randomly choose one from the network
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-              maxNodesPerRack, results, avoidStaleNodes);
+              maxNodesPerRack, results, avoidStaleNodes, storageType);
         }
       } else {
         //otherwise randomly choose one from the network
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+            maxNodesPerRack, results, avoidStaleNodes, storageType);
       }
     }
   }

+ 15 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java

@@ -30,11 +30,11 @@ import org.apache.hadoop.util.LightWeightGSet.SetIterator;
  * the datanodes that store the block.
  */
 class BlocksMap {
-  private static class NodeIterator implements Iterator<DatanodeDescriptor> {
+  private static class StorageIterator implements Iterator<DatanodeStorageInfo> {
     private BlockInfo blockInfo;
     private int nextIdx = 0;
       
-    NodeIterator(BlockInfo blkInfo) {
+    StorageIterator(BlockInfo blkInfo) {
       this.blockInfo = blkInfo;
     }
 
@@ -45,8 +45,8 @@ class BlocksMap {
     }
 
     @Override
-    public DatanodeDescriptor next() {
-      return blockInfo.getDatanode(nextIdx++);
+    public DatanodeStorageInfo next() {
+      return blockInfo.getStorageInfo(nextIdx++);
     }
 
     @Override
@@ -129,18 +129,23 @@ class BlocksMap {
 
   /**
    * Searches for the block in the BlocksMap and 
-   * returns Iterator that iterates through the nodes the block belongs to.
+   * returns {@link Iterable} of the storages the block belongs to.
    */
-  Iterator<DatanodeDescriptor> nodeIterator(Block b) {
-    return nodeIterator(blocks.get(b));
+  Iterable<DatanodeStorageInfo> getStorages(Block b) {
+    return getStorages(blocks.get(b));
   }
 
   /**
    * For a block that has already been retrieved from the BlocksMap
-   * returns Iterator that iterates through the nodes the block belongs to.
+   * returns {@link Iterable} of the storages the block belongs to.
    */
-  Iterator<DatanodeDescriptor> nodeIterator(BlockInfo storedBlock) {
-    return new NodeIterator(storedBlock);
+  Iterable<DatanodeStorageInfo> getStorages(final BlockInfo storedBlock) {
+    return new Iterable<DatanodeStorageInfo>() {
+      @Override
+      public Iterator<DatanodeStorageInfo> iterator() {
+        return new StorageIterator(storedBlock);
+      }
+    };
   }
 
   /** counts number of containing nodes. Better than using iterator. */

+ 210 - 72
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java

@@ -27,6 +27,9 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -48,6 +51,8 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.Time;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Scans the namesystem, scheduling blocks to be cached as appropriate.
  *
@@ -79,25 +84,47 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
   private final long intervalMs;
 
   /**
-   * True if we should rescan immediately, regardless of how much time
-   * elapsed since the previous scan.
+   * The CacheReplicationMonitor (CRM) lock. Used to synchronize starting and
+   * waiting for rescan operations.
    */
-  private boolean rescanImmediately;
+  private final ReentrantLock lock;
 
   /**
-   * The monotonic time at which the current scan started.
+   * Notifies the scan thread that an immediate rescan is needed.
    */
-  private long scanTimeMs;
+  private final Condition doRescan;
 
   /**
-   * Mark status of the current scan.
+   * Notifies waiting threads that a rescan has finished.
    */
-  private boolean mark = false;
+  private final Condition scanFinished;
+
+  /**
+   * Whether there are pending CacheManager operations that necessitate a
+   * CacheReplicationMonitor rescan. Protected by the CRM lock.
+   */
+  private boolean needsRescan = true;
+
+  /**
+   * Whether we are currently doing a rescan. Protected by the CRM lock.
+   */
+  private boolean isScanning = false;
+
+  /**
+   * The number of rescans completed. Used to wait for scans to finish.
+   * Protected by the CacheReplicationMonitor lock.
+   */
+  private long scanCount = 0;
 
   /**
-   * True if this monitor should terminate.
+   * True if this monitor should terminate. Protected by the CRM lock.
    */
-  private boolean shutdown;
+  private boolean shutdown = false;
+
+  /**
+   * Mark status of the current scan.
+   */
+  private boolean mark = false;
 
   /**
    * Cache directives found in the previous scan.
@@ -108,55 +135,74 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
    * Blocks found in the previous scan.
    */
   private long scannedBlocks;
-  
+
   public CacheReplicationMonitor(FSNamesystem namesystem,
-      CacheManager cacheManager, long intervalMs) {
+      CacheManager cacheManager, long intervalMs, ReentrantLock lock) {
     this.namesystem = namesystem;
     this.blockManager = namesystem.getBlockManager();
     this.cacheManager = cacheManager;
     this.cachedBlocks = cacheManager.getCachedBlocks();
     this.intervalMs = intervalMs;
+    this.lock = lock;
+    this.doRescan = this.lock.newCondition();
+    this.scanFinished = this.lock.newCondition();
   }
 
   @Override
   public void run() {
-    shutdown = false;
-    rescanImmediately = true;
-    scanTimeMs = 0;
+    long startTimeMs = 0;
+    Thread.currentThread().setName("CacheReplicationMonitor(" +
+        System.identityHashCode(this) + ")");
     LOG.info("Starting CacheReplicationMonitor with interval " +
              intervalMs + " milliseconds");
     try {
       long curTimeMs = Time.monotonicNow();
       while (true) {
-        synchronized(this) {
+        lock.lock();
+        try {
           while (true) {
             if (shutdown) {
               LOG.info("Shutting down CacheReplicationMonitor");
               return;
             }
-            if (rescanImmediately) {
-              LOG.info("Rescanning on request");
-              rescanImmediately = false;
+            if (needsRescan) {
+              LOG.info("Rescanning because of pending operations");
               break;
             }
-            long delta = (scanTimeMs + intervalMs) - curTimeMs;
+            long delta = (startTimeMs + intervalMs) - curTimeMs;
             if (delta <= 0) {
-              LOG.info("Rescanning after " + (curTimeMs - scanTimeMs) +
+              LOG.info("Rescanning after " + (curTimeMs - startTimeMs) +
                   " milliseconds");
               break;
             }
-            this.wait(delta);
+            doRescan.await(delta, TimeUnit.MILLISECONDS);
             curTimeMs = Time.monotonicNow();
           }
+          isScanning = true;
+          needsRescan = false;
+        } finally {
+          lock.unlock();
         }
-        scanTimeMs = curTimeMs;
+        startTimeMs = curTimeMs;
         mark = !mark;
         rescan();
         curTimeMs = Time.monotonicNow();
+        // Update synchronization-related variables.
+        lock.lock();
+        try {
+          isScanning = false;
+          scanCount++;
+          scanFinished.signalAll();
+        } finally {
+          lock.unlock();
+        }
         LOG.info("Scanned " + scannedDirectives + " directive(s) and " +
-            scannedBlocks + " block(s) in " + (curTimeMs - scanTimeMs) + " " +
+            scannedBlocks + " block(s) in " + (curTimeMs - startTimeMs) + " " +
             "millisecond(s).");
       }
+    } catch (InterruptedException e) {
+      LOG.info("Shutting down CacheReplicationMonitor.");
+      return;
     } catch (Throwable t) {
       LOG.fatal("Thread exiting", t);
       terminate(1, t);
@@ -164,41 +210,80 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
   }
 
   /**
-   * Kick the monitor thread.
-   * 
-   * If it is sleeping, it will wake up and start scanning.
-   * If it is currently scanning, it will finish the scan and immediately do 
-   * another one.
+   * Waits for a rescan to complete. This doesn't guarantee consistency with
+   * pending operations, only relative recency, since it will not force a new
+   * rescan if a rescan is already underway.
+   * <p>
+   * Note that this call will release the FSN lock, so operations before and
+   * after are not atomic.
    */
-  public synchronized void kick() {
-    rescanImmediately = true;
-    this.notifyAll();
+  public void waitForRescanIfNeeded() {
+    Preconditions.checkArgument(!namesystem.hasWriteLock(),
+        "Must not hold the FSN write lock when waiting for a rescan.");
+    Preconditions.checkArgument(lock.isHeldByCurrentThread(),
+        "Must hold the CRM lock when waiting for a rescan.");
+    if (!needsRescan) {
+      return;
+    }
+    // If no scan is already ongoing, mark the CRM as dirty and kick
+    if (!isScanning) {
+      doRescan.signal();
+    }
+    // Wait until the scan finishes and the count advances
+    final long startCount = scanCount;
+    while ((!shutdown) && (startCount >= scanCount)) {
+      try {
+        scanFinished.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for CacheReplicationMonitor"
+            + " rescan", e);
+        break;
+      }
+    }
   }
 
   /**
-   * Shut down and join the monitor thread.
+   * Indicates to the CacheReplicationMonitor that there have been CacheManager
+   * changes that require a rescan.
+   */
+  public void setNeedsRescan() {
+    Preconditions.checkArgument(lock.isHeldByCurrentThread(),
+        "Must hold the CRM lock when setting the needsRescan bit.");
+    this.needsRescan = true;
+  }
+
+  /**
+   * Shut down the monitor thread.
    */
   @Override
   public void close() throws IOException {
-    synchronized(this) {
+    Preconditions.checkArgument(namesystem.hasWriteLock());
+    lock.lock();
+    try {
       if (shutdown) return;
+      // Since we hold both the FSN write lock and the CRM lock here,
+      // we know that the CRM thread cannot be currently modifying
+      // the cache manager state while we're closing it.
+      // Since the CRM thread checks the value of 'shutdown' after waiting
+      // for a lock, we know that the thread will not modify the cache
+      // manager state after this point.
       shutdown = true;
-      this.notifyAll();
-    }
-    try {
-      if (this.isAlive()) {
-        this.join(60000);
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
+      doRescan.signalAll();
+      scanFinished.signalAll();
+    } finally {
+      lock.unlock();
     }
   }
 
-  private void rescan() {
+  private void rescan() throws InterruptedException {
     scannedDirectives = 0;
     scannedBlocks = 0;
     namesystem.writeLock();
     try {
+      if (shutdown) {
+        throw new InterruptedException("CacheReplicationMonitor was " +
+            "shut down.");
+      }
       resetStatistics();
       rescanCacheDirectives();
       rescanCachedBlockMap();
@@ -228,12 +313,14 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       // Reset the directive's statistics
       directive.resetStatistics();
       // Skip processing this entry if it has expired
-      LOG.info("Directive expiry is at " + directive.getExpiryTime());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Directive expiry is at " + directive.getExpiryTime());
+      }
       if (directive.getExpiryTime() > 0 && directive.getExpiryTime() <= now) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Skipping directive id " + directive.getId()
-              + " because it has expired (" + directive.getExpiryTime() + ">="
-              + now);
+              + " because it has expired (" + directive.getExpiryTime() + "<="
+              + now + ")");
         }
         continue;
       }
@@ -280,15 +367,27 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
 
     // Increment the "needed" statistics
     directive.addFilesNeeded(1);
-    long neededTotal = 0;
-    for (BlockInfo blockInfo : blockInfos) {
-      long neededByBlock = 
-          directive.getReplication() * blockInfo.getNumBytes();
-       neededTotal += neededByBlock;
-    }
+    // We don't cache UC blocks, don't add them to the total here
+    long neededTotal = file.computeFileSizeNotIncludingLastUcBlock() *
+        directive.getReplication();
     directive.addBytesNeeded(neededTotal);
 
-    // TODO: Enforce per-pool quotas
+    // The pool's bytesNeeded is incremented as we scan. If the demand
+    // thus far plus the demand of this file would exceed the pool's limit,
+    // do not cache this file.
+    CachePool pool = directive.getPool();
+    if (pool.getBytesNeeded() > pool.getLimit()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("Skipping directive id %d file %s because "
+            + "limit of pool %s would be exceeded (%d > %d)",
+            directive.getId(),
+            file.getFullPathName(),
+            pool.getPoolName(),
+            pool.getBytesNeeded(),
+            pool.getLimit()));
+      }
+      return;
+    }
 
     long cachedTotal = 0;
     for (BlockInfo blockInfo : blockInfos) {
@@ -315,14 +414,21 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
             directive.getReplication()) * blockInfo.getNumBytes();
         cachedTotal += cachedByBlock;
 
-        if (mark != ocblock.getMark()) {
-          // Mark hasn't been set in this scan, so update replication and mark.
+        if ((mark != ocblock.getMark()) ||
+            (ocblock.getReplication() < directive.getReplication())) {
+          //
+          // Overwrite the block's replication and mark in two cases:
+          //
+          // 1. If the mark on the CachedBlock is different from the mark for
+          // this scan, that means the block hasn't been updated during this
+          // scan, and we should overwrite whatever is there, since it is no
+          // longer valid.
+          //
+          // 2. If the replication in the CachedBlock is less than what the
+          // directive asks for, we want to increase the block's replication
+          // field to what the directive asks for.
+          //
           ocblock.setReplicationAndMark(directive.getReplication(), mark);
-        } else {
-          // Mark already set in this scan.  Set replication to highest value in
-          // any CacheDirective that covers this file.
-          ocblock.setReplicationAndMark((short)Math.max(
-              directive.getReplication(), ocblock.getReplication()), mark);
         }
       }
     }
@@ -338,6 +444,36 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
     }
   }
 
+  private String findReasonForNotCaching(CachedBlock cblock, 
+          BlockInfo blockInfo) {
+    if (blockInfo == null) {
+      // Somehow, a cache report with the block arrived, but the block
+      // reports from the DataNode haven't (yet?) described such a block.
+      // Alternately, the NameNode might have invalidated the block, but the
+      // DataNode hasn't caught up.  In any case, we want to tell the DN
+      // to uncache this.
+      return "not tracked by the BlockManager";
+    } else if (!blockInfo.isComplete()) {
+      // When a cached block changes state from complete to some other state
+      // on the DataNode (perhaps because of append), it will begin the
+      // uncaching process.  However, the uncaching process is not
+      // instantaneous, especially if clients have pinned the block.  So
+      // there may be a period of time when incomplete blocks remain cached
+      // on the DataNodes.
+      return "not complete";
+    }  else if (cblock.getReplication() == 0) {
+      // Since 0 is not a valid value for a cache directive's replication
+      // field, seeing a replication of 0 on a CacheBlock means that it
+      // has never been reached by any sweep.
+      return "not needed by any directives";
+    } else if (cblock.getMark() != mark) { 
+      // Although the block was needed in the past, we didn't reach it during
+      // the current sweep.  Therefore, it doesn't need to be cached any more.
+      return "no longer needed by any directives";
+    }
+    return null;
+  }
+
   /**
    * Scan through the cached block map.
    * Any blocks which are under-replicated should be assigned new Datanodes.
@@ -363,11 +499,17 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
           iter.remove();
         }
       }
-      // If the block's mark doesn't match with the mark of this scan, that
-      // means that this block couldn't be reached during this scan.  That means
-      // it doesn't need to be cached any more.
-      int neededCached = (cblock.getMark() != mark) ?
-          0 : cblock.getReplication();
+      BlockInfo blockInfo = blockManager.
+            getStoredBlock(new Block(cblock.getBlockId()));
+      String reason = findReasonForNotCaching(cblock, blockInfo);
+      int neededCached = 0;
+      if (reason != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("not caching " + cblock + " because it is " + reason);
+        }
+      } else {
+        neededCached = cblock.getReplication();
+      }
       int numCached = cached.size();
       if (numCached >= neededCached) {
         // If we have enough replicas, drop all pending cached.
@@ -421,9 +563,6 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
   private void addNewPendingUncached(int neededUncached,
       CachedBlock cachedBlock, List<DatanodeDescriptor> cached,
       List<DatanodeDescriptor> pendingUncached) {
-    if (!cacheManager.isActive()) {
-      return;
-    }
     // Figure out which replicas can be uncached.
     LinkedList<DatanodeDescriptor> possibilities =
         new LinkedList<DatanodeDescriptor>();
@@ -459,16 +598,15 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
   private void addNewPendingCached(int neededCached,
       CachedBlock cachedBlock, List<DatanodeDescriptor> cached,
       List<DatanodeDescriptor> pendingCached) {
-    if (!cacheManager.isActive()) {
-      return;
-    }
     // To figure out which replicas can be cached, we consult the
     // blocksMap.  We don't want to try to cache a corrupt replica, though.
     BlockInfo blockInfo = blockManager.
           getStoredBlock(new Block(cachedBlock.getBlockId()));
     if (blockInfo == null) {
-      LOG.debug("Not caching block " + cachedBlock + " because it " +
-          "was deleted from all DataNodes.");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not caching block " + cachedBlock + " because there " +
+            "is no record of it on the NameNode.");
+      }
       return;
     }
     if (!blockInfo.isComplete()) {

+ 147 - 185
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -18,23 +18,29 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Queue;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.Time;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * This class extends the DatanodeInfo class with ephemeral information (eg
  * health, capacity, what blocks are associated with the Datanode) that is
@@ -43,6 +49,7 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class DatanodeDescriptor extends DatanodeInfo {
+  public static final Log LOG = LogFactory.getLog(DatanodeDescriptor.class);
   public static final DatanodeDescriptor[] EMPTY_ARRAY = {};
 
   // Stores status of decommissioning.
@@ -54,9 +61,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
   @InterfaceStability.Evolving
   public static class BlockTargetPair {
     public final Block block;
-    public final DatanodeDescriptor[] targets;    
+    public final DatanodeStorageInfo[] targets;    
 
-    BlockTargetPair(Block block, DatanodeDescriptor[] targets) {
+    BlockTargetPair(Block block, DatanodeStorageInfo[] targets) {
       this.block = block;
       this.targets = targets;
     }
@@ -99,6 +106,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
+  private final Map<String, DatanodeStorageInfo> storageMap = 
+      new HashMap<String, DatanodeStorageInfo>();
+
   /**
    * A list of CachedBlock objects on this datanode.
    */
@@ -164,37 +174,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
   private long lastCachingDirectiveSentTimeMs;
 
-  /**
-   * Head of the list of blocks on the datanode
-   */
-  private volatile BlockInfo blockList = null;
-  /**
-   * Number of blocks on the datanode
-   */
-  private int numBlocks = 0;
-
   // isAlive == heartbeats.contains(this)
   // This is an optimization, because contains takes O(n) time on Arraylist
   public boolean isAlive = false;
   public boolean needKeyUpdate = false;
 
-  /**
-   * Set to false on any NN failover, and reset to true
-   * whenever a block report is received.
-   */
-  private boolean heartbeatedSinceFailover = false;
-  
-  /**
-   * At startup or at any failover, the DNs in the cluster may
-   * have pending block deletions from a previous incarnation
-   * of the NameNode. Thus, we consider their block contents
-   * stale until we have received a block report. When a DN
-   * is considered stale, any replicas on it are transitively
-   * considered stale. If any block has at least one stale replica,
-   * then no invalidations will be processed for this block.
-   * See HDFS-1972.
-   */
-  private boolean blockContentsStale = true;
   
   // A system administrator can tune the balancer bandwidth parameter
   // (dfs.balance.bandwidthPerSec) dynamically by calling
@@ -213,7 +197,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
 
   /* Variables for maintaining number of blocks scheduled to be written to
-   * this datanode. This count is approximate and might be slightly bigger
+   * this storage. This count is approximate and might be slightly bigger
    * in case of errors (e.g. datanode does not report if an error occurs
    * while writing the block).
    */
@@ -223,9 +207,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
   private int volumeFailures = 0;
   
-  /** Set to false after processing first block report */
-  private boolean firstBlockReport = true;
-  
   /** 
    * When set to true, the node is not in include list and is not allowed
    * to communicate with the namenode
@@ -237,7 +218,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @param nodeID id of the data node
    */
   public DatanodeDescriptor(DatanodeID nodeID) {
-    this(nodeID, 0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
+    super(nodeID);
+    updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
   }
 
   /**
@@ -247,104 +229,60 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
   public DatanodeDescriptor(DatanodeID nodeID, 
                             String networkLocation) {
-    this(nodeID, networkLocation, 0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
-  }
-  
-  /**
-   * DatanodeDescriptor constructor
-   * @param nodeID id of the data node
-   * @param capacity capacity of the data node
-   * @param dfsUsed space used by the data node
-   * @param remaining remaining capacity of the data node
-   * @param bpused space used by the block pool corresponding to this namenode
-   * @param cacheCapacity cache capacity of the data node
-   * @param cacheUsed cache used on the data node
-   * @param xceiverCount # of data transfers at the data node
-   */
-  public DatanodeDescriptor(DatanodeID nodeID, 
-                            long capacity,
-                            long dfsUsed,
-                            long remaining,
-                            long bpused,
-                            long cacheCapacity,
-                            long cacheUsed,
-                            int xceiverCount,
-                            int failedVolumes) {
-    super(nodeID);
-    updateHeartbeat(capacity, dfsUsed, remaining, bpused, cacheCapacity,
-        cacheUsed, xceiverCount, failedVolumes);
+    super(nodeID, networkLocation);
+    updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
   }
 
   /**
-   * DatanodeDescriptor constructor
-   * @param nodeID id of the data node
-   * @param networkLocation location of the data node in network
-   * @param capacity capacity of the data node, including space used by non-dfs
-   * @param dfsUsed the used space by dfs datanode
-   * @param remaining remaining capacity of the data node
-   * @param bpused space used by the block pool corresponding to this namenode
-   * @param cacheCapacity cache capacity of the data node
-   * @param cacheUsed cache used on the data node
-   * @param xceiverCount # of data transfers at the data node
+   * Add data-node to the block. Add block to the head of the list of blocks
+   * belonging to the data-node.
    */
-  public DatanodeDescriptor(DatanodeID nodeID,
-                            String networkLocation,
-                            long capacity,
-                            long dfsUsed,
-                            long remaining,
-                            long bpused,
-                            long cacheCapacity,
-                            long cacheUsed,
-                            int xceiverCount,
-                            int failedVolumes) {
-    super(nodeID, networkLocation);
-    updateHeartbeat(capacity, dfsUsed, remaining, bpused, cacheCapacity,
-        cacheUsed, xceiverCount, failedVolumes);
+  public boolean addBlock(String storageID, BlockInfo b) {
+    DatanodeStorageInfo s = getStorageInfo(storageID);
+    if (s != null) {
+      return s.addBlock(b);
+    }
+    return false;
   }
 
-  /**
-   * Add datanode to the block.
-   * Add block to the head of the list of blocks belonging to the data-node.
-   */
-  public boolean addBlock(BlockInfo b) {
-    if(!b.addNode(this))
-      return false;
-    // add to the head of the data-node list
-    blockList = b.listInsert(blockList, this);
-    numBlocks++;
-    return true;
+  DatanodeStorageInfo getStorageInfo(String storageID) {
+    synchronized (storageMap) {
+      return storageMap.get(storageID);
+    }
   }
-  
-  /**
-   * Remove block from the list of blocks belonging to the data-node.
-   * Remove datanode from the block.
-   */
-  public boolean removeBlock(BlockInfo b) {
-    blockList = b.listRemove(blockList, this);
-    if ( b.removeNode(this) ) {
-      numBlocks--;
-      return true;
-    } else {
-      return false;
+  DatanodeStorageInfo[] getStorageInfos() {
+    synchronized (storageMap) {
+      final Collection<DatanodeStorageInfo> storages = storageMap.values();
+      return storages.toArray(new DatanodeStorageInfo[storages.size()]);
     }
   }
 
   /**
-   * Move block to the head of the list of blocks belonging to the data-node.
-   * @return the index of the head of the blockList
+   * Remove block from the list of blocks belonging to the data-node. Remove
+   * data-node from the block.
    */
-  int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
-    blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
-    return curIndex;
+  boolean removeBlock(BlockInfo b) {
+    int index = b.findStorageInfo(this);
+    // if block exists on this datanode
+    if (index >= 0) {
+      DatanodeStorageInfo s = b.getStorageInfo(index);
+      if (s != null) {
+        return s.removeBlock(b);
+      }
+    }
+    return false;
   }
-
+  
   /**
-   * Used for testing only
-   * @return the head of the blockList
+   * Remove block from the list of blocks belonging to the data-node. Remove
+   * data-node from the block.
    */
-  @VisibleForTesting
-  protected BlockInfo getHead(){
-    return blockList;
+  boolean removeBlock(String storageID, BlockInfo b) {
+    DatanodeStorageInfo s = getStorageInfo(storageID);
+    if (s != null) {
+      return s.removeBlock(b);
+    }
+    return false;
   }
 
   /**
@@ -355,9 +293,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @return the new block
    */
   public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
-    boolean done = removeBlock(oldBlock);
+    int index = oldBlock.findStorageInfo(this);
+    DatanodeStorageInfo s = oldBlock.getStorageInfo(index);
+    boolean done = s.removeBlock(oldBlock);
     assert done : "Old block should belong to the data-node when replacing";
-    done = addBlock(newBlock);
+
+    done = s.addBlock(newBlock);
     assert done : "New block should not belong to the data-node when replacing";
     return newBlock;
   }
@@ -368,7 +309,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     setBlockPoolUsed(0);
     setDfsUsed(0);
     setXceiverCount(0);
-    this.blockList = null;
     this.invalidateBlocks.clear();
     this.volumeFailures = 0;
     // pendingCached, cached, and pendingUncached are protected by the
@@ -392,66 +332,97 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   public int numBlocks() {
-    return numBlocks;
+    int blocks = 0;
+    for (DatanodeStorageInfo entry : getStorageInfos()) {
+      blocks += entry.numBlocks();
+    }
+    return blocks;
   }
 
   /**
    * Updates stats from datanode heartbeat.
    */
-  public void updateHeartbeat(long capacity, long dfsUsed, long remaining,
-      long blockPoolUsed, long cacheCapacity, long cacheUsed, int xceiverCount,
-      int volFailures) {
-    setCapacity(capacity);
-    setRemaining(remaining);
-    setBlockPoolUsed(blockPoolUsed);
-    setDfsUsed(dfsUsed);
+  public void updateHeartbeat(StorageReport[] reports, long cacheCapacity,
+      long cacheUsed, int xceiverCount, int volFailures) {
+    long totalCapacity = 0;
+    long totalRemaining = 0;
+    long totalBlockPoolUsed = 0;
+    long totalDfsUsed = 0;
+
     setCacheCapacity(cacheCapacity);
     setCacheUsed(cacheUsed);
     setXceiverCount(xceiverCount);
     setLastUpdate(Time.now());    
     this.volumeFailures = volFailures;
-    this.heartbeatedSinceFailover = true;
+    for (StorageReport report : reports) {
+      DatanodeStorageInfo storage = storageMap.get(report.getStorageID());
+      if (storage == null) {
+        // This is seen during cluster initialization when the heartbeat
+        // is received before the initial block reports from each storage.
+        storage = updateStorage(new DatanodeStorage(report.getStorageID()));
+      }
+      storage.receivedHeartbeat(report);
+      totalCapacity += report.getCapacity();
+      totalRemaining += report.getRemaining();
+      totalBlockPoolUsed += report.getBlockPoolUsed();
+      totalDfsUsed += report.getDfsUsed();
+    }
     rollBlocksScheduled(getLastUpdate());
+
+    // Update total metrics for the node.
+    setCapacity(totalCapacity);
+    setRemaining(totalRemaining);
+    setBlockPoolUsed(totalBlockPoolUsed);
+    setDfsUsed(totalDfsUsed);
   }
 
-  /**
-   * Iterates over the list of blocks belonging to the datanode.
-   */
-  public static class BlockIterator implements Iterator<BlockInfo> {
-    private BlockInfo current;
-    private DatanodeDescriptor node;
-      
-    BlockIterator(BlockInfo head, DatanodeDescriptor dn) {
-      this.current = head;
-      this.node = dn;
+  private static class BlockIterator implements Iterator<BlockInfo> {
+    private int index = 0;
+    private final List<Iterator<BlockInfo>> iterators;
+    
+    private BlockIterator(final DatanodeStorageInfo... storages) {
+      List<Iterator<BlockInfo>> iterators = new ArrayList<Iterator<BlockInfo>>();
+      for (DatanodeStorageInfo e : storages) {
+        iterators.add(e.getBlockIterator());
+      }
+      this.iterators = Collections.unmodifiableList(iterators);
     }
 
     @Override
     public boolean hasNext() {
-      return current != null;
+      update();
+      return !iterators.isEmpty() && iterators.get(index).hasNext();
     }
 
     @Override
     public BlockInfo next() {
-      BlockInfo res = current;
-      current = current.getNext(current.findDatanode(node));
-      return res;
+      update();
+      return iterators.get(index).next();
     }
-
+    
     @Override
-    public void remove()  {
-      throw new UnsupportedOperationException("Sorry. can't remove.");
+    public void remove() {
+      throw new UnsupportedOperationException("Remove unsupported.");
+    }
+    
+    private void update() {
+      while(index < iterators.size() - 1 && !iterators.get(index).hasNext()) {
+        index++;
+      }
     }
   }
 
-  public Iterator<BlockInfo> getBlockIterator() {
-    return new BlockIterator(this.blockList, this);
+  Iterator<BlockInfo> getBlockIterator() {
+    return new BlockIterator(getStorageInfos());
+  }
+  Iterator<BlockInfo> getBlockIterator(final String storageID) {
+    return new BlockIterator(getStorageInfo(storageID));
   }
 
   /**
    * Store block replication work.
    */
-  void addBlockToBeReplicated(Block block, DatanodeDescriptor[] targets) {
+  void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) {
     assert(block != null && targets != null && targets.length > 0);
     replicateBlocks.offer(new BlockTargetPair(block, targets));
   }
@@ -526,18 +497,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
   public int getBlocksScheduled() {
     return currApproxBlocksScheduled + prevApproxBlocksScheduled;
   }
-  
-  /**
-   * Increments counter for number of blocks scheduled. 
-   */
-  public void incBlocksScheduled() {
+
+  /** Increment the number of blocks scheduled. */
+  void incrementBlocksScheduled() {
     currApproxBlocksScheduled++;
   }
   
-  /**
-   * Decrements counter for number of blocks scheduled.
-   */
-  void decBlocksScheduled() {
+  /** Decrement the number of blocks scheduled. */
+  void decrementBlocksScheduled() {
     if (prevApproxBlocksScheduled > 0) {
       prevApproxBlocksScheduled--;
     } else if (currApproxBlocksScheduled > 0) {
@@ -546,12 +513,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
     // its ok if both counters are zero.
   }
   
-  /**
-   * Adjusts curr and prev number of blocks scheduled every few minutes.
-   */
+  /** Adjusts curr and prev number of blocks scheduled every few minutes. */
   private void rollBlocksScheduled(long now) {
-    if ((now - lastBlocksScheduledRollTime) > 
-        BLOCKS_SCHEDULED_ROLL_INTERVAL) {
+    if (now - lastBlocksScheduledRollTime > BLOCKS_SCHEDULED_ROLL_INTERVAL) {
       prevApproxBlocksScheduled = currApproxBlocksScheduled;
       currApproxBlocksScheduled = 0;
       lastBlocksScheduledRollTime = now;
@@ -647,7 +611,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
   @Override
   public void updateRegInfo(DatanodeID nodeReg) {
     super.updateRegInfo(nodeReg);
-    firstBlockReport = true; // must re-process IBR after re-registration
+    
+    // must re-process IBR after re-registration
+    for(DatanodeStorageInfo storage : getStorageInfos()) {
+      storage.setBlockReportCount(0);
+    }
   }
 
   /**
@@ -664,26 +632,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.bandwidth = bandwidth;
   }
 
-  public boolean areBlockContentsStale() {
-    return blockContentsStale;
-  }
-
-  public void markStaleAfterFailover() {
-    heartbeatedSinceFailover = false;
-    blockContentsStale = true;
-  }
-
-  public void receivedBlockReport() {
-    if (heartbeatedSinceFailover) {
-      blockContentsStale = false;
-    }
-    firstBlockReport = false;
-  }
-  
-  boolean isFirstBlockReport() {
-    return firstBlockReport;
-  }
-
   @Override
   public String dumpDatanode() {
     StringBuilder sb = new StringBuilder(super.dumpDatanode());
@@ -702,6 +650,19 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return sb.toString();
   }
 
+  DatanodeStorageInfo updateStorage(DatanodeStorage s) {
+    synchronized (storageMap) {
+      DatanodeStorageInfo storage = storageMap.get(s.getStorageID());
+      if (storage == null) {
+        LOG.info("Adding new storage ID " + s.getStorageID() +
+                 " for DN " + getXferAddr());
+        storage = new DatanodeStorageInfo(this, s);
+        storageMap.put(s.getStorageID(), storage);
+      }
+      return storage;
+    }
+  }
+
   /**
    * @return   The time at which we last sent caching directives to this 
    *           DataNode, in monotonic milliseconds.
@@ -718,3 +679,4 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.lastCachingDirectiveSentTimeMs = time;
   }
 }
+

+ 60 - 60
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -424,9 +424,13 @@ public class DatanodeManager {
   }
 
 
-  /** Get a datanode descriptor given corresponding storageID */
-  DatanodeDescriptor getDatanode(final String storageID) {
-    return datanodeMap.get(storageID);
+  /** Get a datanode descriptor given corresponding DatanodeUUID */
+  DatanodeDescriptor getDatanode(final String datanodeUuid) {
+    if (datanodeUuid == null) {
+      return null;
+    }
+
+    return datanodeMap.get(datanodeUuid);
   }
 
   /**
@@ -438,7 +442,7 @@ public class DatanodeManager {
    */
   public DatanodeDescriptor getDatanode(DatanodeID nodeID
       ) throws UnregisteredNodeException {
-    final DatanodeDescriptor node = getDatanode(nodeID.getStorageID());
+    final DatanodeDescriptor node = getDatanode(nodeID.getDatanodeUuid());
     if (node == null) 
       return null;
     if (!node.getXferAddr().equals(nodeID.getXferAddr())) {
@@ -451,6 +455,20 @@ public class DatanodeManager {
     return node;
   }
 
+  public DatanodeStorageInfo[] getDatanodeStorageInfos(
+      DatanodeID[] datanodeID, String[] storageIDs)
+          throws UnregisteredNodeException {
+    if (datanodeID.length == 0) {
+      return null;
+    }
+    final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length];
+    for(int i = 0; i < datanodeID.length; i++) {
+      final DatanodeDescriptor dd = getDatanode(datanodeID[i]);
+      storages[i] = dd.getStorageInfo(storageIDs[i]);
+    }
+    return storages; 
+  }
+
   /** Prints information about all datanodes. */
   void datanodeDump(final PrintWriter out) {
     synchronized (datanodeMap) {
@@ -528,7 +546,7 @@ public class DatanodeManager {
     // remove  from host2DatanodeMap the datanodeDescriptor removed
     // from datanodeMap before adding node to host2DatanodeMap.
     synchronized(datanodeMap) {
-      host2DatanodeMap.remove(datanodeMap.put(node.getStorageID(), node));
+      host2DatanodeMap.remove(datanodeMap.put(node.getDatanodeUuid(), node));
     }
 
     networktopology.add(node); // may throw InvalidTopologyException
@@ -543,7 +561,7 @@ public class DatanodeManager {
 
   /** Physically remove node from datanodeMap. */
   private void wipeDatanode(final DatanodeID node) {
-    final String key = node.getStorageID();
+    final String key = node.getDatanodeUuid();
     synchronized (datanodeMap) {
       host2DatanodeMap.remove(datanodeMap.remove(key));
     }
@@ -705,8 +723,10 @@ public class DatanodeManager {
   /** Start decommissioning the specified datanode. */
   private void startDecommission(DatanodeDescriptor node) {
     if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
-      LOG.info("Start Decommissioning " + node + " with " + 
-          node.numBlocks() +  " blocks");
+      for (DatanodeStorageInfo storage : node.getStorageInfos()) {
+        LOG.info("Start Decommissioning " + node + " " + storage
+            + " with " + storage.numBlocks() + " blocks");
+      }
       heartbeatManager.startDecommission(node);
       node.decommissioningStatus.setStartTime(now());
       
@@ -728,24 +748,6 @@ public class DatanodeManager {
     }
   }
 
-  /**
-   * Generate new storage ID.
-   * 
-   * @return unique storage ID
-   * 
-   * Note: that collisions are still possible if somebody will try 
-   * to bring in a data storage from a different cluster.
-   */
-  private String newStorageID() {
-    String newID = null;
-    while(newID == null) {
-      newID = "DS" + Integer.toString(DFSUtil.getRandom().nextInt());
-      if (datanodeMap.get(newID) != null)
-        newID = null;
-    }
-    return newID;
-  }
-
   /**
    * Register the given datanode with the namenode. NB: the given
    * registration is mutated and given back to the datanode.
@@ -784,9 +786,9 @@ public class DatanodeManager {
       }
         
       NameNode.stateChangeLog.info("BLOCK* registerDatanode: from "
-          + nodeReg + " storage " + nodeReg.getStorageID());
+          + nodeReg + " storage " + nodeReg.getDatanodeUuid());
   
-      DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
+      DatanodeDescriptor nodeS = getDatanode(nodeReg.getDatanodeUuid());
       DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
           nodeReg.getIpAddr(), nodeReg.getXferPort());
         
@@ -821,7 +823,7 @@ public class DatanodeManager {
          */        
           NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS
               + " is replaced by " + nodeReg + " with the same storageID "
-              + nodeReg.getStorageID());
+              + nodeReg.getDatanodeUuid());
         }
         
         boolean success = false;
@@ -853,20 +855,8 @@ public class DatanodeManager {
           }
         }
         return;
-      } 
-  
-      // this is a new datanode serving a new data storage
-      if ("".equals(nodeReg.getStorageID())) {
-        // this data storage has never been registered
-        // it is either empty or was created by pre-storageID version of DFS
-        nodeReg.setStorageID(newStorageID());
-        if (NameNode.stateChangeLog.isDebugEnabled()) {
-          NameNode.stateChangeLog.debug(
-              "BLOCK* NameSystem.registerDatanode: "
-              + "new storageID " + nodeReg.getStorageID() + " assigned.");
-        }
       }
-      
+
       DatanodeDescriptor nodeDescr 
         = new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK);
       boolean success = false;
@@ -1234,10 +1224,10 @@ public class DatanodeManager {
 
   /** Handle heartbeat from datanodes. */
   public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
-      final String blockPoolId,
-      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      long cacheCapacity, long cacheUsed, int xceiverCount, int maxTransfers,
-      int failedVolumes) throws IOException {
+      StorageReport[] reports, final String blockPoolId,
+      long cacheCapacity, long cacheUsed, int xceiverCount, 
+      int maxTransfers, int failedVolumes
+      ) throws IOException {
     synchronized (heartbeatManager) {
       synchronized (datanodeMap) {
         DatanodeDescriptor nodeinfo = null;
@@ -1257,9 +1247,9 @@ public class DatanodeManager {
           return new DatanodeCommand[]{RegisterCommand.REGISTER};
         }
 
-        heartbeatManager.updateHeartbeat(nodeinfo, capacity, dfsUsed,
-            remaining, blockPoolUsed, cacheCapacity, cacheUsed, xceiverCount,
-            failedVolumes);
+        heartbeatManager.updateHeartbeat(nodeinfo, reports,
+                                         cacheCapacity, cacheUsed,
+                                         xceiverCount, failedVolumes);
 
         // If we are in safemode, do not send back any recovery / replication
         // requests. Don't even drain the existing queue of work.
@@ -1274,32 +1264,32 @@ public class DatanodeManager {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
           for (BlockInfoUnderConstruction b : blocks) {
-            DatanodeDescriptor[] expectedLocations = b.getExpectedLocations();
+            final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
-            List<DatanodeDescriptor> recoveryLocations =
-                new ArrayList<DatanodeDescriptor>(expectedLocations.length);
-            for (int i = 0; i < expectedLocations.length; i++) {
-              if (!expectedLocations[i].isStale(this.staleInterval)) {
-                recoveryLocations.add(expectedLocations[i]);
+            final List<DatanodeStorageInfo> recoveryLocations =
+                new ArrayList<DatanodeStorageInfo>(storages.length);
+            for (int i = 0; i < storages.length; i++) {
+              if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
+                recoveryLocations.add(storages[i]);
               }
             }
             // If we only get 1 replica after eliminating stale nodes, then choose all
             // replicas for recovery and let the primary data node handle failures.
             if (recoveryLocations.size() > 1) {
-              if (recoveryLocations.size() != expectedLocations.length) {
+              if (recoveryLocations.size() != storages.length) {
                 LOG.info("Skipped stale nodes for recovery : " +
-                    (expectedLocations.length - recoveryLocations.size()));
+                    (storages.length - recoveryLocations.size()));
               }
               brCommand.add(new RecoveringBlock(
                   new ExtendedBlock(blockPoolId, b),
-                  recoveryLocations.toArray(new DatanodeDescriptor[recoveryLocations.size()]),
+                  DatanodeStorageInfo.toDatanodeInfos(recoveryLocations),
                   b.getBlockRecoveryId()));
             } else {
               // If too many replicas are stale, then choose all replicas to participate
               // in block recovery.
               brCommand.add(new RecoveringBlock(
                   new ExtendedBlock(blockPoolId, b),
-                  expectedLocations,
+                  DatanodeStorageInfo.toDatanodeInfos(storages),
                   b.getBlockRecoveryId()));
             }
           }
@@ -1416,7 +1406,9 @@ public class DatanodeManager {
     LOG.info("Marking all datandoes as stale");
     synchronized (datanodeMap) {
       for (DatanodeDescriptor dn : datanodeMap.values()) {
-        dn.markStaleAfterFailover();
+        for(DatanodeStorageInfo storage : dn.getStorageInfos()) {
+          storage.markStaleAfterFailover();
+        }
       }
     }
   }
@@ -1451,7 +1443,15 @@ public class DatanodeManager {
     return getClass().getSimpleName() + ": " + host2DatanodeMap;
   }
 
+  public void clearPendingCachingCommands() {
+    for (DatanodeDescriptor dn : datanodeMap.values()) {
+      dn.getPendingCached().clear();
+      dn.getPendingUncached().clear();
+    }
+  }
+
   public void setShouldSendCachingCommands(boolean shouldSendCachingCommands) {
     this.shouldSendCachingCommands = shouldSendCachingCommands;
   }
 }
+

+ 288 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java

@@ -0,0 +1,288 @@
+/**
+ * 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.blockmanagement;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+
+/**
+ * A Datanode has one or more storages. A storage in the Datanode is represented
+ * by this class.
+ */
+public class DatanodeStorageInfo {
+  public static final DatanodeStorageInfo[] EMPTY_ARRAY = {};
+
+  public static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) {
+    return toDatanodeInfos(Arrays.asList(storages));
+  }
+  static DatanodeInfo[] toDatanodeInfos(List<DatanodeStorageInfo> storages) {
+    final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()];
+    for(int i = 0; i < storages.size(); i++) {
+      datanodes[i] = storages.get(i).getDatanodeDescriptor();
+    }
+    return datanodes;
+  }
+
+  static DatanodeDescriptor[] toDatanodeDescriptors(
+      DatanodeStorageInfo[] storages) {
+    DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length];
+    for (int i = 0; i < storages.length; ++i) {
+      datanodes[i] = storages[i].getDatanodeDescriptor();
+    }
+    return datanodes;
+  }
+
+  public static String[] toStorageIDs(DatanodeStorageInfo[] storages) {
+    String[] storageIDs = new String[storages.length];
+    for(int i = 0; i < storageIDs.length; i++) {
+      storageIDs[i] = storages[i].getStorageID();
+    }
+    return storageIDs;
+  }
+
+  public static StorageType[] toStorageTypes(DatanodeStorageInfo[] storages) {
+    StorageType[] storageTypes = new StorageType[storages.length];
+    for(int i = 0; i < storageTypes.length; i++) {
+      storageTypes[i] = storages[i].getStorageType();
+    }
+    return storageTypes;
+  }
+
+  /**
+   * Iterates over the list of blocks belonging to the data-node.
+   */
+  class BlockIterator implements Iterator<BlockInfo> {
+    private BlockInfo current;
+
+    BlockIterator(BlockInfo head) {
+      this.current = head;
+    }
+
+    public boolean hasNext() {
+      return current != null;
+    }
+
+    public BlockInfo next() {
+      BlockInfo res = current;
+      current = current.getNext(current.findStorageInfo(DatanodeStorageInfo.this));
+      return res;
+    }
+
+    public void remove() {
+      throw new UnsupportedOperationException("Sorry. can't remove.");
+    }
+  }
+
+  private final DatanodeDescriptor dn;
+  private final String storageID;
+  private final StorageType storageType;
+  private final State state;
+
+  private long capacity;
+  private long dfsUsed;
+  private long remaining;
+  private long blockPoolUsed;
+
+  private volatile BlockInfo blockList = null;
+  private int numBlocks = 0;
+
+  /** The number of block reports received */
+  private int blockReportCount = 0;
+
+  /**
+   * Set to false on any NN failover, and reset to true
+   * whenever a block report is received.
+   */
+  private boolean heartbeatedSinceFailover = false;
+
+  /**
+   * At startup or at failover, the storages in the cluster may have pending
+   * block deletions from a previous incarnation of the NameNode. The block
+   * contents are considered as stale until a block report is received. When a
+   * storage is considered as stale, the replicas on it are also considered as
+   * stale. If any block has at least one stale replica, then no invalidations
+   * will be processed for this block. See HDFS-1972.
+   */
+  private boolean blockContentsStale = true;
+
+  DatanodeStorageInfo(DatanodeDescriptor dn, DatanodeStorage s) {
+    this.dn = dn;
+    this.storageID = s.getStorageID();
+    this.storageType = s.getStorageType();
+    this.state = s.getState();
+  }
+
+  int getBlockReportCount() {
+    return blockReportCount;
+  }
+
+  void setBlockReportCount(int blockReportCount) {
+    this.blockReportCount = blockReportCount;
+  }
+
+  boolean areBlockContentsStale() {
+    return blockContentsStale;
+  }
+
+  void markStaleAfterFailover() {
+    heartbeatedSinceFailover = false;
+    blockContentsStale = true;
+  }
+
+  void receivedHeartbeat(StorageReport report) {
+    updateState(report);
+    heartbeatedSinceFailover = true;
+  }
+
+  void receivedBlockReport() {
+    if (heartbeatedSinceFailover) {
+      blockContentsStale = false;
+    }
+    blockReportCount++;
+  }
+
+  @VisibleForTesting
+  public void setUtilizationForTesting(long capacity, long dfsUsed,
+                      long remaining, long blockPoolUsed) {
+    this.capacity = capacity;
+    this.dfsUsed = dfsUsed;
+    this.remaining = remaining;
+    this.blockPoolUsed = blockPoolUsed;
+  }
+  
+  State getState() {
+    return this.state;
+  }
+  
+  String getStorageID() {
+    return storageID;
+  }
+
+  StorageType getStorageType() {
+    return storageType;
+  }
+
+  long getCapacity() {
+    return capacity;
+  }
+
+  long getDfsUsed() {
+    return dfsUsed;
+  }
+
+  long getRemaining() {
+    return remaining;
+  }
+
+  long getBlockPoolUsed() {
+    return blockPoolUsed;
+  }
+
+  boolean addBlock(BlockInfo b) {
+    if(!b.addStorage(this))
+      return false;
+    // add to the head of the data-node list
+    blockList = b.listInsert(blockList, this);
+    numBlocks++;
+    return true;
+  }
+
+  boolean removeBlock(BlockInfo b) {
+    blockList = b.listRemove(blockList, this);
+    if (b.removeStorage(this)) {
+      numBlocks--;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  int numBlocks() {
+    return numBlocks;
+  }
+  
+  Iterator<BlockInfo> getBlockIterator() {
+    return new BlockIterator(blockList);
+
+  }
+
+  /**
+   * Move block to the head of the list of blocks belonging to the data-node.
+   * @return the index of the head of the blockList
+   */
+  int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
+    blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
+    return curIndex;
+  }
+
+  /**
+   * Used for testing only
+   * @return the head of the blockList
+   */
+  @VisibleForTesting
+  BlockInfo getBlockListHeadForTesting(){
+    return blockList;
+  }
+
+  void updateState(StorageReport r) {
+    capacity = r.getCapacity();
+    dfsUsed = r.getDfsUsed();
+    remaining = r.getRemaining();
+    blockPoolUsed = r.getBlockPoolUsed();
+  }
+
+  public DatanodeDescriptor getDatanodeDescriptor() {
+    return dn;
+  }
+
+  /** Increment the number of blocks scheduled for each given storage */ 
+  public static void incrementBlocksScheduled(DatanodeStorageInfo... storages) {
+    for (DatanodeStorageInfo s : storages) {
+      s.getDatanodeDescriptor().incrementBlocksScheduled();
+    }
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    } else if (obj == null || !(obj instanceof DatanodeStorageInfo)) {
+      return false;
+    }
+    final DatanodeStorageInfo that = (DatanodeStorageInfo)obj;
+    return this.storageID.equals(that.storageID);
+  }
+
+  @Override
+  public int hashCode() {
+    return storageID.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "[" + storageType + "]" + storageID + ":" + state;
+  }
+}

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Time;
 
@@ -181,7 +182,7 @@ class HeartbeatManager implements DatanodeStatistics {
       addDatanode(d);
 
       //update its timestamp
-      d.updateHeartbeat(0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
+      d.updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
     }
   }
 
@@ -203,11 +204,11 @@ class HeartbeatManager implements DatanodeStatistics {
   }
 
   synchronized void updateHeartbeat(final DatanodeDescriptor node,
-      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      long cacheCapacity, long cacheUsed, int xceiverCount, int failedVolumes) {
+      StorageReport[] reports, long cacheCapacity, long cacheUsed,
+      int xceiverCount, int failedVolumes) {
     stats.subtract(node);
-    node.updateHeartbeat(capacity, dfsUsed, remaining, blockPoolUsed,
-        cacheCapacity, cacheUsed, xceiverCount, failedVolumes);
+    node.updateHeartbeat(reports, cacheCapacity, cacheUsed,
+      xceiverCount, failedVolumes);
     stats.add(node);
   }
 
@@ -358,3 +359,4 @@ class HeartbeatManager implements DatanodeStatistics {
     }
   }
 }
+

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

@@ -78,10 +78,10 @@ class InvalidateBlocks {
    */
   synchronized void add(final Block block, final DatanodeInfo datanode,
       final boolean log) {
-    LightWeightHashSet<Block> set = node2blocks.get(datanode.getStorageID());
+    LightWeightHashSet<Block> set = node2blocks.get(datanode.getDatanodeUuid());
     if (set == null) {
       set = new LightWeightHashSet<Block>();
-      node2blocks.put(datanode.getStorageID(), set);
+      node2blocks.put(datanode.getDatanodeUuid(), set);
     }
     if (set.add(block)) {
       numBlocks++;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java

@@ -34,5 +34,5 @@ public interface MutableBlockCollection extends BlockCollection {
    * and set the locations.
    */
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
-      DatanodeDescriptor[] locations) throws IOException;
+      DatanodeStorageInfo[] storages) throws IOException;
 }

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java

@@ -42,11 +42,13 @@ class PendingDataNodeMessages {
   static class ReportedBlockInfo {
     private final Block block;
     private final DatanodeDescriptor dn;
+    private final String storageID;
     private final ReplicaState reportedState;
 
-    ReportedBlockInfo(DatanodeDescriptor dn, Block block,
+    ReportedBlockInfo(DatanodeDescriptor dn, String storageID, Block block,
         ReplicaState reportedState) {
       this.dn = dn;
+      this.storageID = storageID;
       this.block = block;
       this.reportedState = reportedState;
     }
@@ -58,6 +60,10 @@ class PendingDataNodeMessages {
     DatanodeDescriptor getNode() {
       return dn;
     }
+    
+    String getStorageID() {
+      return storageID;
+    }
 
     ReplicaState getReportedState() {
       return reportedState;
@@ -70,11 +76,11 @@ class PendingDataNodeMessages {
     }
   }
   
-  void enqueueReportedBlock(DatanodeDescriptor dn, Block block,
+  void enqueueReportedBlock(DatanodeDescriptor dn, String storageID, Block block,
       ReplicaState reportedState) {
     block = new Block(block);
     getBlockQueue(block).add(
-        new ReportedBlockInfo(dn, block, reportedState));
+        new ReportedBlockInfo(dn, storageID, block, reportedState));
     count++;
   }
   

+ 15 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -117,6 +117,18 @@ public class JspHelper {
       return 0;
     }
   }
+  
+  /**
+   * convenience method for canonicalizing host name.
+   * @param addr name:port or name 
+   * @return canonicalized host name
+   */
+   public static String canonicalize(String addr) {
+    // default port 1 is supplied to allow addr without port.
+    // the port will be ignored.
+    return NetUtils.createSocketAddr(addr, 1).getAddress()
+           .getCanonicalHostName();
+  }
 
   /**
    * A helper class that generates the correct URL for different schema.
@@ -124,10 +136,11 @@ public class JspHelper {
    */
   public static final class Url {
     public static String authority(String scheme, DatanodeID d) {
+      String fqdn = canonicalize(d.getIpAddr());
       if (scheme.equals("http")) {
-        return d.getInfoAddr();
+        return fqdn + ":" + d.getInfoPort();
       } else if (scheme.equals("https")) {
-        return d.getInfoSecureAddr();
+        return fqdn + ":" + d.getInfoSecurePort();
       } else {
         throw new IllegalArgumentException("Unknown scheme:" + scheme);
       }

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -236,6 +236,8 @@ public abstract class Storage extends StorageInfo {
     final boolean useLock;        // flag to enable storage lock
     final StorageDirType dirType; // storage dir type
     FileLock lock;                // storage lock
+
+    private String storageUuid = null;      // Storage directory identifier.
     
     public StorageDirectory(File dir) {
       // default dirType is null
@@ -246,6 +248,14 @@ public abstract class Storage extends StorageInfo {
       this(dir, dirType, true);
     }
     
+    public void setStorageUuid(String storageUuid) {
+      this.storageUuid = storageUuid;
+    }
+
+    public String getStorageUuid() {
+      return storageUuid;
+    }
+
     /**
      * Constructor
      * @param dir directory corresponding to the storage

+ 36 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

@@ -27,6 +27,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -147,7 +148,7 @@ class BPOfferService {
     return false;
   }
   
-  String getBlockPoolId() {
+  synchronized String getBlockPoolId() {
     if (bpNSInfo != null) {
       return bpNSInfo.getBlockPoolID();
     } else {
@@ -160,31 +161,32 @@ class BPOfferService {
   synchronized NamespaceInfo getNamespaceInfo() {
     return bpNSInfo;
   }
-  
+
   @Override
-  public String toString() {
+  public synchronized String toString() {
     if (bpNSInfo == null) {
       // If we haven't yet connected to our NN, we don't yet know our
       // own block pool ID.
       // If _none_ of the block pools have connected yet, we don't even
-      // know the storage ID of this DN.
-      String storageId = dn.getStorageId();
-      if (storageId == null || "".equals(storageId)) {
-        storageId = "unknown";
+      // know the DatanodeID ID of this DN.
+      String datanodeUuid = dn.getDatanodeUuid();
+
+      if (datanodeUuid == null || datanodeUuid.isEmpty()) {
+        datanodeUuid = "unassigned";
       }
-      return "Block pool <registering> (storage id " + storageId +
-        ")";
+      return "Block pool <registering> (Datanode Uuid " + datanodeUuid + ")";
     } else {
       return "Block pool " + getBlockPoolId() +
-        " (storage id " + dn.getStorageId() +
-        ")";
+          " (Datanode Uuid " + dn.getDatanodeUuid() +
+          ")";
     }
   }
   
-  void reportBadBlocks(ExtendedBlock block) {
+  void reportBadBlocks(ExtendedBlock block,
+                       String storageUuid, StorageType storageType) {
     checkBlock(block);
     for (BPServiceActor actor : bpServices) {
-      actor.reportBadBlocks(block);
+      actor.reportBadBlocks(block, storageUuid, storageType);
     }
   }
   
@@ -193,7 +195,8 @@ class BPOfferService {
    * till namenode is informed before responding with success to the
    * client? For now we don't.
    */
-  void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
+  void notifyNamenodeReceivedBlock(
+      ExtendedBlock block, String delHint, String storageUuid) {
     checkBlock(block);
     checkDelHint(delHint);
     ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
@@ -202,7 +205,7 @@ class BPOfferService {
         delHint);
 
     for (BPServiceActor actor : bpServices) {
-      actor.notifyNamenodeBlockImmediately(bInfo);
+      actor.notifyNamenodeBlockImmediately(bInfo, storageUuid);
     }
   }
 
@@ -219,23 +222,23 @@ class BPOfferService {
         "delHint is null");
   }
 
-  void notifyNamenodeDeletedBlock(ExtendedBlock block) {
+  void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) {
     checkBlock(block);
     ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
        block.getLocalBlock(), BlockStatus.DELETED_BLOCK, null);
     
     for (BPServiceActor actor : bpServices) {
-      actor.notifyNamenodeDeletedBlock(bInfo);
+      actor.notifyNamenodeDeletedBlock(bInfo, storageUuid);
     }
   }
   
-  void notifyNamenodeReceivingBlock(ExtendedBlock block) {
+  void notifyNamenodeReceivingBlock(ExtendedBlock block, String storageUuid) {
     checkBlock(block);
     ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
        block.getLocalBlock(), BlockStatus.RECEIVING_BLOCK, null);
     
     for (BPServiceActor actor : bpServices) {
-      actor.notifyNamenodeBlockImmediately(bInfo);
+      actor.notifyNamenodeBlockImmediately(bInfo, storageUuid);
     }
   }
 
@@ -274,12 +277,22 @@ class BPOfferService {
   synchronized void verifyAndSetNamespaceInfo(NamespaceInfo nsInfo) throws IOException {
     if (this.bpNSInfo == null) {
       this.bpNSInfo = nsInfo;
-      
+      boolean success = false;
+
       // Now that we know the namespace ID, etc, we can pass this to the DN.
       // The DN can now initialize its local storage if we are the
       // first BP to handshake, etc.
-      dn.initBlockPool(this);
-      return;
+      try {
+        dn.initBlockPool(this);
+        success = true;
+      } finally {
+        if (!success) {
+          // The datanode failed to initialize the BP. We need to reset
+          // the namespace info so that other BPService actors still have
+          // a chance to set it, and re-initialize the datanode.
+          this.bpNSInfo = null;
+        }
+      }
     } else {
       checkNSEquality(bpNSInfo.getBlockPoolID(), nsInfo.getBlockPoolID(),
           "Blockpool ID");
@@ -328,7 +341,7 @@ class BPOfferService {
     }
   }
 
-  synchronized DatanodeRegistration createRegistration() {
+  synchronized DatanodeRegistration createRegistration() throws IOException {
     Preconditions.checkState(bpNSInfo != null,
         "getRegistration() can only be called after initial handshake");
     return dn.createBPRegistration(bpNSInfo);

+ 212 - 79
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -22,7 +22,7 @@ import static org.apache.hadoop.util.Time.now;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
-import java.net.URI;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -52,7 +53,6 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionUtil;
@@ -100,9 +100,9 @@ class BPServiceActor implements Runnable {
    * keyed by block ID, contains the pending changes which have yet to be
    * reported to the NN. Access should be synchronized on this object.
    */
-  private final Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR 
-    = Maps.newHashMap();
-  
+  private final Map<String, PerStoragePendingIncrementalBR>
+      pendingIncrementalBRperStorage = Maps.newHashMap();
+
   private volatile int pendingReceivedRequests = 0;
   private volatile boolean shouldServiceRun = true;
   private final DataNode dn;
@@ -244,12 +244,15 @@ class BPServiceActor implements Runnable {
     resetBlockReportTime = true; // reset future BRs for randomness
   }
 
-  void reportBadBlocks(ExtendedBlock block) {
+  void reportBadBlocks(ExtendedBlock block,
+      String storageUuid, StorageType storageType) {
     if (bpRegistration == null) {
       return;
     }
     DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) };
-    LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) }; 
+    String[] uuids = { storageUuid };
+    StorageType[] types = { storageType };
+    LocatedBlock[] blocks = { new LocatedBlock(block, dnArr, uuids, types) };
     
     try {
       bpNamenode.reportBadBlocks(blocks);  
@@ -263,69 +266,122 @@ class BPServiceActor implements Runnable {
   }
   
   /**
-   * Report received blocks and delete hints to the Namenode
-   * 
+   * Report received blocks and delete hints to the Namenode for each
+   * storage.
+   *
    * @throws IOException
    */
   private void reportReceivedDeletedBlocks() throws IOException {
 
-    // check if there are newly received blocks
-    ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
-    synchronized (pendingIncrementalBR) {
-      int numBlocks = pendingIncrementalBR.size();
-      if (numBlocks > 0) {
-        //
-        // Send newly-received and deleted blockids to namenode
-        //
-        receivedAndDeletedBlockArray = pendingIncrementalBR
-            .values().toArray(new ReceivedDeletedBlockInfo[numBlocks]);
+    // Generate a list of the pending reports for each storage under the lock
+    ArrayList<StorageReceivedDeletedBlocks> reports =
+        new ArrayList<StorageReceivedDeletedBlocks>(pendingIncrementalBRperStorage.size());
+    synchronized (pendingIncrementalBRperStorage) {
+      for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
+           pendingIncrementalBRperStorage.entrySet()) {
+        final String storageUuid = entry.getKey();
+        final PerStoragePendingIncrementalBR perStorageMap = entry.getValue();
+
+        if (perStorageMap.getBlockInfoCount() > 0) {
+          // Send newly-received and deleted blockids to namenode
+          ReceivedDeletedBlockInfo[] rdbi = perStorageMap.dequeueBlockInfos();
+          pendingReceivedRequests =
+              (pendingReceivedRequests > rdbi.length ?
+                  (pendingReceivedRequests - rdbi.length) : 0);
+          reports.add(new StorageReceivedDeletedBlocks(storageUuid, rdbi));
+        }
       }
-      pendingIncrementalBR.clear();
     }
-    if (receivedAndDeletedBlockArray != null) {
-      StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
-          bpRegistration.getStorageID(), receivedAndDeletedBlockArray) };
-      boolean success = false;
-      try {
-        bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
-            report);
-        success = true;
-      } finally {
-        synchronized (pendingIncrementalBR) {
-          if (!success) {
+
+    if (reports.size() == 0) {
+      // Nothing new to report.
+      return;
+    }
+
+    // Send incremental block reports to the Namenode outside the lock
+    boolean success = false;
+    try {
+      bpNamenode.blockReceivedAndDeleted(bpRegistration,
+          bpos.getBlockPoolId(),
+          reports.toArray(new StorageReceivedDeletedBlocks[reports.size()]));
+      success = true;
+    } finally {
+      if (!success) {
+        synchronized (pendingIncrementalBRperStorage) {
+          for (StorageReceivedDeletedBlocks report : reports) {
             // If we didn't succeed in sending the report, put all of the
-            // blocks back onto our queue, but only in the case where we didn't
-            // put something newer in the meantime.
-            for (ReceivedDeletedBlockInfo rdbi : receivedAndDeletedBlockArray) {
-              if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
-                pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
-              }
-            }
+            // blocks back onto our queue, but only in the case where we
+            // didn't put something newer in the meantime.
+            PerStoragePendingIncrementalBR perStorageMap =
+                pendingIncrementalBRperStorage.get(report.getStorageID());
+            pendingReceivedRequests +=
+                perStorageMap.putMissingBlockInfos(report.getBlocks());
           }
-          pendingReceivedRequests = pendingIncrementalBR.size();
         }
       }
     }
   }
 
+  /**
+   * Retrieve the incremental BR state for a given storage UUID
+   * @param storageUuid
+   * @return
+   */
+  private PerStoragePendingIncrementalBR getIncrementalBRMapForStorage(
+      String storageUuid) {
+    PerStoragePendingIncrementalBR mapForStorage =
+        pendingIncrementalBRperStorage.get(storageUuid);
+
+    if (mapForStorage == null) {
+      // This is the first time we are adding incremental BR state for
+      // this storage so create a new map. This is required once per
+      // storage, per service actor.
+      mapForStorage = new PerStoragePendingIncrementalBR();
+      pendingIncrementalBRperStorage.put(storageUuid, mapForStorage);
+    }
+
+    return mapForStorage;
+  }
+
+  /**
+   * Add a blockInfo for notification to NameNode. If another entry
+   * exists for the same block it is removed.
+   *
+   * Caller must synchronize access using pendingIncrementalBRperStorage.
+   * @param bInfo
+   * @param storageUuid
+   */
+  void addPendingReplicationBlockInfo(ReceivedDeletedBlockInfo bInfo,
+      String storageUuid) {
+    // Make sure another entry for the same block is first removed.
+    // There may only be one such entry.
+    for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
+          pendingIncrementalBRperStorage.entrySet()) {
+      if (entry.getValue().removeBlockInfo(bInfo)) {
+        break;
+      }
+    }
+    getIncrementalBRMapForStorage(storageUuid).putBlockInfo(bInfo);
+  }
+
   /*
    * Informing the name node could take a long long time! Should we wait
    * till namenode is informed before responding with success to the
    * client? For now we don't.
    */
-  void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) {
-    synchronized (pendingIncrementalBR) {
-      pendingIncrementalBR.put(
-          bInfo.getBlock().getBlockId(), bInfo);
+  void notifyNamenodeBlockImmediately(
+      ReceivedDeletedBlockInfo bInfo, String storageUuid) {
+    synchronized (pendingIncrementalBRperStorage) {
+      addPendingReplicationBlockInfo(bInfo, storageUuid);
       pendingReceivedRequests++;
-      pendingIncrementalBR.notifyAll();
+      pendingIncrementalBRperStorage.notifyAll();
     }
   }
 
-  void notifyNamenodeDeletedBlock(ReceivedDeletedBlockInfo bInfo) {
-    synchronized (pendingIncrementalBR) {
-      pendingIncrementalBR.put(
-          bInfo.getBlock().getBlockId(), bInfo);
+  void notifyNamenodeDeletedBlock(
+      ReceivedDeletedBlockInfo bInfo, String storageUuid) {
+    synchronized (pendingIncrementalBRperStorage) {
+      addPendingReplicationBlockInfo(bInfo, storageUuid);
     }
   }
 
@@ -334,13 +390,13 @@ class BPServiceActor implements Runnable {
    */
   @VisibleForTesting
   void triggerBlockReportForTests() {
-    synchronized (pendingIncrementalBR) {
+    synchronized (pendingIncrementalBRperStorage) {
       lastBlockReport = 0;
       lastHeartbeat = 0;
-      pendingIncrementalBR.notifyAll();
+      pendingIncrementalBRperStorage.notifyAll();
       while (lastBlockReport == 0) {
         try {
-          pendingIncrementalBR.wait(100);
+          pendingIncrementalBRperStorage.wait(100);
         } catch (InterruptedException e) {
           return;
         }
@@ -350,12 +406,12 @@ class BPServiceActor implements Runnable {
   
   @VisibleForTesting
   void triggerHeartbeatForTests() {
-    synchronized (pendingIncrementalBR) {
+    synchronized (pendingIncrementalBRperStorage) {
       lastHeartbeat = 0;
-      pendingIncrementalBR.notifyAll();
+      pendingIncrementalBRperStorage.notifyAll();
       while (lastHeartbeat == 0) {
         try {
-          pendingIncrementalBR.wait(100);
+          pendingIncrementalBRperStorage.wait(100);
         } catch (InterruptedException e) {
           return;
         }
@@ -365,13 +421,13 @@ class BPServiceActor implements Runnable {
 
   @VisibleForTesting
   void triggerDeletionReportForTests() {
-    synchronized (pendingIncrementalBR) {
+    synchronized (pendingIncrementalBRperStorage) {
       lastDeletedReport = 0;
-      pendingIncrementalBR.notifyAll();
+      pendingIncrementalBRperStorage.notifyAll();
 
       while (lastDeletedReport == 0) {
         try {
-          pendingIncrementalBR.wait(100);
+          pendingIncrementalBRperStorage.wait(100);
         } catch (InterruptedException e) {
           return;
         }
@@ -395,23 +451,38 @@ class BPServiceActor implements Runnable {
       // a FINALIZED one.
       reportReceivedDeletedBlocks();
 
+      // Send one block report per known storage.
+
       // Create block report
       long brCreateStartTime = now();
-      BlockListAsLongs bReport = dn.getFSDataset().getBlockReport(
-          bpos.getBlockPoolId());
+      long totalBlockCount = 0;
+
+      Map<DatanodeStorage, BlockListAsLongs> perVolumeBlockLists =
+          dn.getFSDataset().getBlockReports(bpos.getBlockPoolId());
 
       // Send block report
       long brSendStartTime = now();
-      StorageBlockReport[] report = { new StorageBlockReport(
-          new DatanodeStorage(bpRegistration.getStorageID()),
-          bReport.getBlockListAsLongs()) };
-      cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), report);
+      StorageBlockReport[] reports =
+          new StorageBlockReport[perVolumeBlockLists.size()];
+
+      int i = 0;
+      for(Map.Entry<DatanodeStorage, BlockListAsLongs> kvPair : perVolumeBlockLists.entrySet()) {
+        DatanodeStorage dnStorage = kvPair.getKey();
+        BlockListAsLongs blockList = kvPair.getValue();
+        totalBlockCount += blockList.getNumberOfBlocks();
+
+        reports[i++] =
+            new StorageBlockReport(
+              dnStorage, blockList.getBlockListAsLongs());
+      }
+
+      cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), reports);
 
       // Log the block report processing stats from Datanode perspective
       long brSendCost = now() - brSendStartTime;
       long brCreateCost = brSendStartTime - brCreateStartTime;
       dn.getMetrics().addBlockReport(brSendCost);
-      LOG.info("BlockReport of " + bReport.getNumberOfBlocks()
+      LOG.info("BlockReport of " + totalBlockCount
           + " blocks took " + brCreateCost + " msec to generate and "
           + brSendCost + " msecs for RPC and NN processing");
 
@@ -466,17 +537,15 @@ class BPServiceActor implements Runnable {
   }
   
   HeartbeatResponse sendHeartBeat() throws IOException {
+    StorageReport[] reports =
+        dn.getFSDataset().getStorageReports(bpos.getBlockPoolId());
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Sending heartbeat from service actor: " + this);
-    }
-    // reports number of failed volumes
-    StorageReport[] report = { new StorageReport(bpRegistration.getStorageID(),
-        false,
-        dn.getFSDataset().getCapacity(),
-        dn.getFSDataset().getDfsUsed(),
-        dn.getFSDataset().getRemaining(),
-        dn.getFSDataset().getBlockPoolUsed(bpos.getBlockPoolId())) };
-    return bpNamenode.sendHeartbeat(bpRegistration, report,
+      LOG.debug("Sending heartbeat with " + reports.length +
+                " storage reports from service actor: " + this);
+    }
+
+    return bpNamenode.sendHeartbeat(bpRegistration,
+        reports,
         dn.getFSDataset().getCacheCapacity(),
         dn.getFSDataset().getCacheUsed(),
         dn.getXmitsInProgress(),
@@ -496,9 +565,9 @@ class BPServiceActor implements Runnable {
   }
   
   private String formatThreadName() {
-    Collection<URI> dataDirs = DataNode.getStorageDirs(dn.getConf());
-    return "DataNode: [" +
-      StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "] " +
+    Collection<StorageLocation> dataDirs =
+        DataNode.getStorageLocations(dn.getConf());
+    return "DataNode: [" + dataDirs.toString() + "] " +
       " heartbeating to " + nnAddr;
   }
   
@@ -608,10 +677,10 @@ class BPServiceActor implements Runnable {
         //
         long waitTime = dnConf.heartBeatInterval - 
         (Time.now() - lastHeartbeat);
-        synchronized(pendingIncrementalBR) {
+        synchronized(pendingIncrementalBRperStorage) {
           if (waitTime > 0 && pendingReceivedRequests == 0) {
             try {
-              pendingIncrementalBR.wait(waitTime);
+              pendingIncrementalBRperStorage.wait(waitTime);
             } catch (InterruptedException ie) {
               LOG.warn("BPOfferService for " + this + " interrupted");
             }
@@ -782,4 +851,68 @@ class BPServiceActor implements Runnable {
     }
   }
 
+  private static class PerStoragePendingIncrementalBR {
+    private Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR =
+        Maps.newHashMap();
+
+    /**
+     * Return the number of blocks on this storage that have pending
+     * incremental block reports.
+     * @return
+     */
+    int getBlockInfoCount() {
+      return pendingIncrementalBR.size();
+    }
+
+    /**
+     * Dequeue and return all pending incremental block report state.
+     * @return
+     */
+    ReceivedDeletedBlockInfo[] dequeueBlockInfos() {
+      ReceivedDeletedBlockInfo[] blockInfos =
+          pendingIncrementalBR.values().toArray(
+              new ReceivedDeletedBlockInfo[getBlockInfoCount()]);
+
+      pendingIncrementalBR.clear();
+      return blockInfos;
+    }
+
+    /**
+     * Add blocks from blockArray to pendingIncrementalBR, unless the
+     * block already exists in pendingIncrementalBR.
+     * @param blockArray list of blocks to add.
+     * @return the number of missing blocks that we added.
+     */
+    int putMissingBlockInfos(ReceivedDeletedBlockInfo[] blockArray) {
+      int blocksPut = 0;
+      for (ReceivedDeletedBlockInfo rdbi : blockArray) {
+        if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
+          pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
+          ++blocksPut;
+        }
+      }
+      return blocksPut;
+    }
+
+    /**
+     * Add pending incremental block report for a single block.
+     * @param blockID
+     * @param blockInfo
+     */
+    void putBlockInfo(ReceivedDeletedBlockInfo blockInfo) {
+      pendingIncrementalBR.put(blockInfo.getBlock().getBlockId(), blockInfo);
+    }
+
+    /**
+     * Remove pending incremental block report for a single block if it
+     * exists.
+     *
+     * @param blockInfo
+     * @return true if a report was removed, false if no report existed for
+     *         the given block.
+     */
+    boolean removeBlockInfo(ReceivedDeletedBlockInfo blockInfo) {
+      return (pendingIncrementalBR.remove(blockInfo.getBlock().getBlockId()) != null);
+    }
+  }
 }

+ 26 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java

@@ -21,10 +21,13 @@ import java.io.BufferedInputStream;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
+import java.io.EOFException;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
@@ -67,7 +70,29 @@ public class BlockMetadataHeader {
     return checksum;
   }
 
- 
+  /**
+   * Read the header without changing the position of the FileChannel.
+   *
+   * @param fc The FileChannel to read.
+   * @return the Metadata Header.
+   * @throws IOException on error.
+   */
+  public static BlockMetadataHeader preadHeader(FileChannel fc)
+      throws IOException {
+    byte arr[] = new byte[2 + DataChecksum.HEADER_LEN];
+    ByteBuffer buf = ByteBuffer.wrap(arr);
+
+    while (buf.hasRemaining()) {
+      if (fc.read(buf, 0) <= 0) {
+        throw new EOFException("unexpected EOF while reading " +
+            "metadata file header");
+      }
+    }
+    short version = (short)((arr[0] << 8) | (arr[1] & 0xff));
+    DataChecksum dataChecksum = DataChecksum.newDataChecksum(arr, 2);
+    return new BlockMetadataHeader(version, dataChecksum);
+  }
+
   /**
    * This reads all the fields till the beginning of checksum.
    * @param in 

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

@@ -187,7 +187,7 @@ class BlockPoolSliceScanner {
         + hours + " hours for block pool " + bpid);
 
     // get the list of blocks and arrange them in random order
-    List<Block> arr = dataset.getFinalizedBlocks(blockPoolId);
+    List<FinalizedReplica> arr = dataset.getFinalizedBlocks(blockPoolId);
     Collections.shuffle(arr);
     
     long scanTime = -1;

+ 13 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -162,7 +162,8 @@ class BlockReceiver implements Closeable {
         switch (stage) {
         case PIPELINE_SETUP_CREATE:
           replicaInfo = datanode.data.createRbw(block);
-          datanode.notifyNamenodeReceivingBlock(block);
+          datanode.notifyNamenodeReceivingBlock(
+              block, replicaInfo.getStorageUuid());
           break;
         case PIPELINE_SETUP_STREAMING_RECOVERY:
           replicaInfo = datanode.data.recoverRbw(
@@ -176,7 +177,8 @@ class BlockReceiver implements Closeable {
                 block.getLocalBlock());
           }
           block.setGenerationStamp(newGs);
-          datanode.notifyNamenodeReceivingBlock(block);
+          datanode.notifyNamenodeReceivingBlock(
+              block, replicaInfo.getStorageUuid());
           break;
         case PIPELINE_SETUP_APPEND_RECOVERY:
           replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
@@ -185,7 +187,8 @@ class BlockReceiver implements Closeable {
                 block.getLocalBlock());
           }
           block.setGenerationStamp(newGs);
-          datanode.notifyNamenodeReceivingBlock(block);
+          datanode.notifyNamenodeReceivingBlock(
+              block, replicaInfo.getStorageUuid());
           break;
         case TRANSFER_RBW:
         case TRANSFER_FINALIZED:
@@ -252,6 +255,10 @@ class BlockReceiver implements Closeable {
   /** Return the datanode object. */
   DataNode getDataNode() {return datanode;}
 
+  String getStorageUuid() {
+    return replicaInfo.getStorageUuid();
+  }
+
   /**
    * close files.
    */
@@ -1073,14 +1080,15 @@ class BlockReceiver implements Closeable {
           : 0;
       block.setNumBytes(replicaInfo.getNumBytes());
       datanode.data.finalizeBlock(block);
-      datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
+      datanode.closeBlock(
+          block, DataNode.EMPTY_DEL_HINT, replicaInfo.getStorageUuid());
       if (ClientTraceLog.isInfoEnabled() && isClient) {
         long offset = 0;
         DatanodeRegistration dnR = datanode.getDNRegistrationForBP(block
             .getBlockPoolId());
         ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT, inAddr,
             myAddr, block.getNumBytes(), "HDFS_WRITE", clientname, offset,
-            dnR.getStorageID(), block, endTime - startTime));
+            dnR.getDatanodeUuid(), block, endTime - startTime));
       } else {
         LOG.info("Received " + block + " size " + block.getNumBytes()
             + " from " + inAddr);

+ 24 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/CachingStrategy.java

@@ -21,8 +21,8 @@ package org.apache.hadoop.hdfs.server.datanode;
  * The caching strategy we should use for an HDFS read or write operation.
  */
 public class CachingStrategy {
-  private Boolean dropBehind; // null = use server defaults
-  private Long readahead; // null = use server defaults
+  private final Boolean dropBehind; // null = use server defaults
+  private final Long readahead; // null = use server defaults
   
   public static CachingStrategy newDefaultStrategy() {
     return new CachingStrategy(null, null);
@@ -32,8 +32,28 @@ public class CachingStrategy {
     return new CachingStrategy(true, null);
   }
 
-  public CachingStrategy duplicate() {
-    return new CachingStrategy(this.dropBehind, this.readahead);
+  public static class Builder {
+    private Boolean dropBehind;
+    private Long readahead;
+
+    public Builder(CachingStrategy prev) {
+      this.dropBehind = prev.dropBehind;
+      this.readahead = prev.readahead;
+    }
+
+    public Builder setDropBehind(Boolean dropBehind) {
+      this.dropBehind = dropBehind;
+      return this;
+    }
+
+    public Builder setReadahead(Long readahead) {
+      this.readahead = readahead;
+      return this;
+    }
+
+    public CachingStrategy build() {
+      return new CachingStrategy(dropBehind, readahead);
+    }
   }
 
   public CachingStrategy(Boolean dropBehind, Long readahead) {
@@ -45,18 +65,10 @@ public class CachingStrategy {
     return dropBehind;
   }
   
-  public void setDropBehind(Boolean dropBehind) {
-    this.dropBehind = dropBehind;
-  }
-  
   public Long getReadahead() {
     return readahead;
   }
 
-  public void setReadahead(Long readahead) {
-    this.readahead = readahead;
-  }
-
   public String toString() {
     return "CachingStrategy(dropBehind=" + dropBehind +
         ", readahead=" + readahead + ")";

+ 172 - 123
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -17,10 +17,40 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.protobuf.BlockingService;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.SocketChannel;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.management.ObjectName;
+
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -38,21 +68,42 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.net.DomainPeerServer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
-import org.apache.hadoop.hdfs.protocol.*;
-import org.apache.hadoop.hdfs.protocol.datatransfer.*;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
-import org.apache.hadoop.hdfs.protocolPB.*;
-import org.apache.hadoop.hdfs.security.token.block.*;
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -61,7 +112,11 @@ import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMetho
 import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
-import org.apache.hadoop.hdfs.server.protocol.*;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.http.HttpConfig;
@@ -84,23 +139,21 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.util.*;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.JvmPauseMonitor;
+import org.apache.hadoop.util.ServicePlugin;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 
-import java.io.*;
-import java.net.*;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.SocketChannel;
-import java.security.PrivilegedExceptionAction;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.management.ObjectName;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import static org.apache.hadoop.util.ExitUtil.terminate;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.BlockingService;
 
 /**********************************************************
  * DataNode is a class (and program) that stores a set of
@@ -209,7 +262,7 @@ public class DataNode extends Configured
   private JvmPauseMonitor pauseMonitor;
 
   private SecureResources secureResources = null;
-  private AbstractList<File> dataDirs;
+  private List<StorageLocation> dataDirs;
   private Configuration conf;
   private final long maxNumberOfBlocksToLog;
 
@@ -219,21 +272,12 @@ public class DataNode extends Configured
   private final boolean getHdfsBlockLocationsEnabled;
   private ObjectName dataNodeInfoBeanName;
 
-  /**
-   * Create the DataNode given a configuration and an array of dataDirs.
-   * 'dataDirs' is where the blocks are stored.
-   */
-  DataNode(final Configuration conf, 
-           final AbstractList<File> dataDirs) throws IOException {
-    this(conf, dataDirs, null);
-  }
-  
   /**
    * Create the DataNode given a configuration, an array of dataDirs,
    * and a namenode proxy
    */
-  DataNode(final Configuration conf, 
-           final AbstractList<File> dataDirs,
+  DataNode(final Configuration conf,
+           final List<StorageLocation> dataDirs,
            final SecureResources resources) throws IOException {
     super(conf);
     this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
@@ -494,7 +538,7 @@ public class DataNode extends Configured
       directoryScanner.start();
     } else {
       LOG.info("Periodic Directory Tree Verification scan is disabled because " +
-               reason);
+                   reason);
     }
   }
   
@@ -566,10 +610,11 @@ public class DataNode extends Configured
   }
   
   // calls specific to BP
-  protected void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
+  protected void notifyNamenodeReceivedBlock(
+      ExtendedBlock block, String delHint, String storageUuid) {
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if(bpos != null) {
-      bpos.notifyNamenodeReceivedBlock(block, delHint); 
+      bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
     } else {
       LOG.error("Cannot find BPOfferService for reporting block received for bpid="
           + block.getBlockPoolId());
@@ -577,10 +622,11 @@ public class DataNode extends Configured
   }
   
   // calls specific to BP
-  protected void notifyNamenodeReceivingBlock(ExtendedBlock block) {
+  protected void notifyNamenodeReceivingBlock(
+      ExtendedBlock block, String storageUuid) {
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if(bpos != null) {
-      bpos.notifyNamenodeReceivingBlock(block); 
+      bpos.notifyNamenodeReceivingBlock(block, storageUuid);
     } else {
       LOG.error("Cannot find BPOfferService for reporting block receiving for bpid="
           + block.getBlockPoolId());
@@ -588,10 +634,10 @@ public class DataNode extends Configured
   }
   
   /** Notify the corresponding namenode to delete the block. */
-  public void notifyNamenodeDeletedBlock(ExtendedBlock block) {
+  public void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) {
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if (bpos != null) {
-      bpos.notifyNamenodeDeletedBlock(block);
+      bpos.notifyNamenodeDeletedBlock(block, storageUuid);
     } else {
       LOG.error("Cannot find BPOfferService for reporting block deleted for bpid="
           + block.getBlockPoolId());
@@ -603,7 +649,9 @@ public class DataNode extends Configured
    */
   public void reportBadBlocks(ExtendedBlock block) throws IOException{
     BPOfferService bpos = getBPOSForBlock(block);
-    bpos.reportBadBlocks(block);
+    FsVolumeSpi volume = getFSDataset().getVolume(block);
+    bpos.reportBadBlocks(
+        block, volume.getStorageID(), volume.getStorageType());
   }
 
   /**
@@ -675,7 +723,7 @@ public class DataNode extends Configured
    * @throws IOException
    */
   void startDataNode(Configuration conf, 
-                     AbstractList<File> dataDirs,
+                     List<StorageLocation> dataDirs,
                     // DatanodeProtocol namenode,
                      SecureResources resources
                      ) throws IOException {
@@ -736,19 +784,40 @@ public class DataNode extends Configured
     readaheadPool = ReadaheadPool.getInstance();
   }
   
+  public static String generateUuid() {
+    return UUID.randomUUID().toString();
+  }
+
+  /**
+   * Verify that the DatanodeUuid has been initialized. If this is a new
+   * datanode then we generate a new Datanode Uuid and persist it to disk.
+   *
+   * @throws IOException
+   */
+  private synchronized void checkDatanodeUuid() throws IOException {
+    if (storage.getDatanodeUuid() == null) {
+      storage.setDatanodeUuid(generateUuid());
+      storage.writeAll();
+      LOG.info("Generated and persisted new Datanode UUID " +
+               storage.getDatanodeUuid());
+    }
+  }
+
   /**
    * Create a DatanodeRegistration for a specific block pool.
    * @param nsInfo the namespace info from the first part of the NN handshake
    */
-  DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo) {
+  DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo)
+      throws IOException {
     StorageInfo storageInfo = storage.getBPStorage(nsInfo.getBlockPoolID());
     if (storageInfo == null) {
       // it's null in the case of SimulatedDataSet
       storageInfo = new StorageInfo(nsInfo);
     }
+
     DatanodeID dnId = new DatanodeID(
         streamingAddr.getAddress().getHostAddress(), hostName, 
-        getStorageId(), getXferPort(), getInfoPort(),
+        storage.getDatanodeUuid(), getXferPort(), getInfoPort(),
             infoSecurePort, getIpcPort());
     return new DatanodeRegistration(dnId, storageInfo, 
         new ExportedBlockKeys(), VersionInfo.getVersion());
@@ -767,16 +836,10 @@ public class DataNode extends Configured
       id = bpRegistration;
     }
 
-    if (storage.getStorageID().equals("")) {
-      // This is a fresh datanode, persist the NN-provided storage ID
-      storage.setStorageID(bpRegistration.getStorageID());
-      storage.writeAll();
-      LOG.info("New storage id " + bpRegistration.getStorageID()
-          + " is assigned to data-node " + bpRegistration);
-    } else if(!storage.getStorageID().equals(bpRegistration.getStorageID())) {
-      throw new IOException("Inconsistent storage IDs. Name-node returned "
-          + bpRegistration.getStorageID() 
-          + ". Expecting " + storage.getStorageID());
+    if(!storage.getDatanodeUuid().equals(bpRegistration.getDatanodeUuid())) {
+      throw new IOException("Inconsistent Datanode IDs. Name-node returned "
+          + bpRegistration.getDatanodeUuid()
+          + ". Expecting " + storage.getDatanodeUuid());
     }
     
     registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
@@ -897,9 +960,12 @@ public class DataNode extends Configured
       final StorageInfo bpStorage = storage.getBPStorage(bpid);
       LOG.info("Setting up storage: nsid=" + bpStorage.getNamespaceID()
           + ";bpid=" + bpid + ";lv=" + storage.getLayoutVersion()
-          + ";nsInfo=" + nsInfo);
+          + ";nsInfo=" + nsInfo + ";dnuuid=" + storage.getDatanodeUuid());
     }
 
+    // If this is a newly formatted DataNode then assign a new DatanodeUuid.
+    checkDatanodeUuid();
+
     synchronized(this)  {
       if (data == null) {
         data = factory.newInstance(this, storage, conf);
@@ -924,10 +990,6 @@ public class DataNode extends Configured
     return streamingAddr.getPort();
   }
   
-  String getStorageId() {
-    return storage.getStorageID();
-  }
-
   /**
    * @return name useful for logging
    */
@@ -1013,34 +1075,6 @@ public class DataNode extends Configured
     return metrics;
   }
   
-  public static void setNewStorageID(DatanodeID dnId) {
-    LOG.info("Datanode is " + dnId);
-    dnId.setStorageID(createNewStorageId(dnId.getXferPort()));
-  }
-  
-  /**
-   * @return a unique storage ID of form "DS-randInt-ipaddr-port-timestamp"
-   */
-  static String createNewStorageId(int port) {
-    // It is unlikely that we will create a non-unique storage ID
-    // for the following reasons:
-    // a) SecureRandom is a cryptographically strong random number generator
-    // b) IP addresses will likely differ on different hosts
-    // c) DataNode xfer ports will differ on the same host
-    // d) StorageIDs will likely be generated at different times (in ms)
-    // A conflict requires that all four conditions are violated.
-    // NB: The format of this string can be changed in the future without
-    // requiring that old SotrageIDs be updated.
-    String ip = "unknownIP";
-    try {
-      ip = DNS.getDefaultIP("default");
-    } catch (UnknownHostException ignored) {
-      LOG.warn("Could not find an IP address for the \"default\" inteface.");
-    }
-    int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
-    return "DS-" + rand + "-" + ip + "-" + port + "-" + Time.now();
-  }
-  
   /** Ensure the authentication method is kerberos */
   private void checkKerberosAuthMethod(String msg) throws IOException {
     // User invoking the call must be same as the datanode user
@@ -1370,8 +1404,10 @@ public class DataNode extends Configured
     // Check if NN recorded length matches on-disk length 
     long onDiskLength = data.getLength(block);
     if (block.getNumBytes() > onDiskLength) {
+      FsVolumeSpi volume = getFSDataset().getVolume(block);
       // Shorter on-disk len indicates corruption so report NN the corrupt block
-      bpos.reportBadBlocks(block);
+      bpos.reportBadBlocks(
+          block, volume.getStorageID(), volume.getStorageType());
       LOG.warn("Can't replicate block " + block
           + " because on-disk length " + onDiskLength 
           + " is shorter than NameNode recorded length " + block.getNumBytes());
@@ -1635,11 +1671,11 @@ public class DataNode extends Configured
    * @param block
    * @param delHint
    */
-  void closeBlock(ExtendedBlock block, String delHint) {
+  void closeBlock(ExtendedBlock block, String delHint, String storageUuid) {
     metrics.incrBlocksWritten();
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if(bpos != null) {
-      bpos.notifyNamenodeReceivedBlock(block, delHint);
+      bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
     } else {
       LOG.warn("Cannot find BPOfferService for reporting block received for bpid="
           + block.getBlockPoolId());
@@ -1703,17 +1739,32 @@ public class DataNode extends Configured
       printUsage(System.err);
       return null;
     }
-    Collection<URI> dataDirs = getStorageDirs(conf);
+    Collection<StorageLocation> dataLocations = getStorageLocations(conf);
     UserGroupInformation.setConfiguration(conf);
     SecurityUtil.login(conf, DFS_DATANODE_KEYTAB_FILE_KEY,
         DFS_DATANODE_USER_NAME_KEY);
-    return makeInstance(dataDirs, conf, resources);
+    return makeInstance(dataLocations, conf, resources);
   }
 
-  static Collection<URI> getStorageDirs(Configuration conf) {
-    Collection<String> dirNames =
-      conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
-    return Util.stringCollectionAsURIs(dirNames);
+  public static List<StorageLocation> getStorageLocations(Configuration conf) {
+    Collection<String> rawLocations =
+        conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
+    List<StorageLocation> locations =
+        new ArrayList<StorageLocation>(rawLocations.size());
+
+    for(String locationString : rawLocations) {
+      final StorageLocation location;
+      try {
+        location = StorageLocation.parse(locationString);
+      } catch (IOException ioe) {
+        throw new IllegalArgumentException("Failed to parse conf property "
+            + DFS_DATANODE_DATA_DIR_KEY + ": " + locationString, ioe);
+      }
+
+      locations.add(location);
+    }
+
+    return locations;
   }
 
   /** Instantiate & Start a single datanode daemon and wait for it to finish.
@@ -1779,57 +1830,52 @@ public class DataNode extends Configured
    * no directory from this directory list can be created.
    * @throws IOException
    */
-  static DataNode makeInstance(Collection<URI> dataDirs, Configuration conf,
-      SecureResources resources) throws IOException {
+  static DataNode makeInstance(Collection<StorageLocation> dataDirs,
+      Configuration conf, SecureResources resources) throws IOException {
     LocalFileSystem localFS = FileSystem.getLocal(conf);
     FsPermission permission = new FsPermission(
         conf.get(DFS_DATANODE_DATA_DIR_PERMISSION_KEY,
                  DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT));
     DataNodeDiskChecker dataNodeDiskChecker =
         new DataNodeDiskChecker(permission);
-    ArrayList<File> dirs =
-        getDataDirsFromURIs(dataDirs, localFS, dataNodeDiskChecker);
+    List<StorageLocation> locations =
+        checkStorageLocations(dataDirs, localFS, dataNodeDiskChecker);
     DefaultMetricsSystem.initialize("DataNode");
 
-    assert dirs.size() > 0 : "number of data directories should be > 0";
-    return new DataNode(conf, dirs, resources);
+    assert locations.size() > 0 : "number of data directories should be > 0";
+    return new DataNode(conf, locations, resources);
   }
 
   // DataNode ctor expects AbstractList instead of List or Collection...
-  static ArrayList<File> getDataDirsFromURIs(Collection<URI> dataDirs,
+  static List<StorageLocation> checkStorageLocations(
+      Collection<StorageLocation> dataDirs,
       LocalFileSystem localFS, DataNodeDiskChecker dataNodeDiskChecker)
           throws IOException {
-    ArrayList<File> dirs = new ArrayList<File>();
+    ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
     StringBuilder invalidDirs = new StringBuilder();
-    for (URI dirURI : dataDirs) {
-      if (!"file".equalsIgnoreCase(dirURI.getScheme())) {
-        LOG.warn("Unsupported URI schema in " + dirURI + ". Ignoring ...");
-        invalidDirs.append("\"").append(dirURI).append("\" ");
-        continue;
-      }
-      // drop any (illegal) authority in the URI for backwards compatibility
-      File dir = new File(dirURI.getPath());
+    for (StorageLocation location : dataDirs) {
+      final URI uri = location.getUri();
       try {
-        dataNodeDiskChecker.checkDir(localFS, new Path(dir.toURI()));
-        dirs.add(dir);
+        dataNodeDiskChecker.checkDir(localFS, new Path(uri));
+        locations.add(location);
       } catch (IOException ioe) {
         LOG.warn("Invalid " + DFS_DATANODE_DATA_DIR_KEY + " "
-            + dir + " : ", ioe);
-        invalidDirs.append("\"").append(dirURI.getPath()).append("\" ");
+            + location.getFile() + " : ", ioe);
+        invalidDirs.append("\"").append(uri.getPath()).append("\" ");
       }
     }
-    if (dirs.size() == 0) {
+    if (locations.size() == 0) {
       throw new IOException("All directories in "
           + DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
           + invalidDirs);
     }
-    return dirs;
+    return locations;
   }
 
   @Override
   public String toString() {
     return "DataNode{data=" + data + ", localName='" + getDisplayName()
-        + "', storageID='" + getStorageId() + "', xmitsInProgress="
+        + "', datanodeUuid='" + storage.getDatanodeUuid() + "', xmitsInProgress="
         + xmitsInProgress.get() + "}";
   }
 
@@ -1883,7 +1929,6 @@ public class DataNode extends Configured
   }
 
   /**
-   * This method is used for testing. 
    * Examples are adding and deleting blocks directly.
    * The most common usage will be when the data node's storage is simulated.
    * 
@@ -1983,7 +2028,7 @@ public class DataNode extends Configured
     ExtendedBlock newBlock = new ExtendedBlock(oldBlock);
     newBlock.setGenerationStamp(recoveryId);
     newBlock.setNumBytes(newLength);
-    notifyNamenodeReceivedBlock(newBlock, "");
+    notifyNamenodeReceivedBlock(newBlock, "", storageID);
     return storageID;
   }
 
@@ -2443,6 +2488,10 @@ public class DataNode extends Configured
     return dnConf;
   }
 
+  public String getDatanodeUuid() {
+    return id == null ? null : id.getDatanodeUuid();
+  }
+
   boolean shouldRun() {
     return shouldRun;
   }

+ 69 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -24,13 +24,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileLock;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
+import java.util.*;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -50,6 +44,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
@@ -71,8 +66,13 @@ public class DataStorage extends Storage {
   public final static String STORAGE_DIR_FINALIZED = "finalized";
   public final static String STORAGE_DIR_TMP = "tmp";
 
-  /** Unique storage ID. {@see DataNode#createNewStorageId(int)} for details */
-  private String storageID;
+  /**
+   * Datanode UUID that this storage is currently attached to. This
+   *  is the same as the legacy StorageID for datanodes that were
+   *  upgraded from a pre-UUID version. For compatibility with prior
+   *  versions of Datanodes we cannot make this field a UUID.
+   */
+  private String datanodeUuid = null;
 
   // Flag to ensure we only initialize storage once
   private boolean initialized = false;
@@ -84,33 +84,29 @@ public class DataStorage extends Storage {
 
   DataStorage() {
     super(NodeType.DATA_NODE);
-    storageID = "";
   }
   
   public StorageInfo getBPStorage(String bpid) {
     return bpStorageMap.get(bpid);
   }
   
-  public DataStorage(StorageInfo storageInfo, String strgID) {
+  public DataStorage(StorageInfo storageInfo) {
     super(NodeType.DATA_NODE, storageInfo);
-    this.storageID = strgID;
   }
 
-  /** @return storage ID. */
-  public synchronized String getStorageID() {
-    return storageID;
+  public synchronized String getDatanodeUuid() {
+    return datanodeUuid;
   }
-  
-  synchronized void setStorageID(String newStorageID) {
-    this.storageID = newStorageID;
+
+  public synchronized void setDatanodeUuid(String newDatanodeUuid) {
+    this.datanodeUuid = newDatanodeUuid;
   }
 
   /** Create an ID for this storage. */
-  public synchronized void createStorageID(int datanodePort) {
-    if (storageID != null && !storageID.isEmpty()) {
-      return;
+  public synchronized void createStorageID(StorageDirectory sd) {
+    if (sd.getStorageUuid() == null) {
+      sd.setStorageUuid(DatanodeStorage.generateUuid());
     }
-    storageID = DataNode.createNewStorageId(datanodePort);
   }
   
   /**
@@ -128,7 +124,8 @@ public class DataStorage extends Storage {
    * @throws IOException
    */
   synchronized void recoverTransitionRead(DataNode datanode,
-      NamespaceInfo nsInfo, Collection<File> dataDirs, StartupOption startOpt)
+      NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
+      StartupOption startOpt)
       throws IOException {
     if (initialized) {
       // DN storage has been initialized, no need to do anything
@@ -144,8 +141,8 @@ public class DataStorage extends Storage {
     // Format and recover.
     this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
     ArrayList<StorageState> dataDirStates = new ArrayList<StorageState>(dataDirs.size());
-    for(Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
-      File dataDir = it.next();
+    for(Iterator<StorageLocation> it = dataDirs.iterator(); it.hasNext();) {
+      File dataDir = it.next().getFile();
       StorageDirectory sd = new StorageDirectory(dataDir);
       StorageState curState;
       try {
@@ -162,7 +159,7 @@ public class DataStorage extends Storage {
         case NOT_FORMATTED: // format
           LOG.info("Storage directory " + dataDir + " is not formatted");
           LOG.info("Formatting ...");
-          format(sd, nsInfo);
+          format(sd, nsInfo, datanode.getDatanodeUuid());
           break;
         default:  // recovery part is common
           sd.doRecover(curState);
@@ -191,11 +188,9 @@ public class DataStorage extends Storage {
       doTransition(datanode, getStorageDir(idx), nsInfo, startOpt);
       assert this.getLayoutVersion() == nsInfo.getLayoutVersion() :
         "Data-node and name-node layout versions must be the same.";
+      createStorageID(getStorageDir(idx));
     }
     
-    // make sure we have storage id set - if not - generate new one
-    createStorageID(datanode.getXferPort());
-    
     // 3. Update all storages. Some of them might have just been formatted.
     this.writeAll();
     
@@ -214,14 +209,14 @@ public class DataStorage extends Storage {
    * @throws IOException on error
    */
   void recoverTransitionRead(DataNode datanode, String bpID, NamespaceInfo nsInfo,
-      Collection<File> dataDirs, StartupOption startOpt) throws IOException {
+      Collection<StorageLocation> dataDirs, StartupOption startOpt) throws IOException {
     // First ensure datanode level format/snapshot/rollback is completed
     recoverTransitionRead(datanode, nsInfo, dataDirs, startOpt);
-    
+
     // Create list of storage directories for the block pool
     Collection<File> bpDataDirs = new ArrayList<File>();
-    for(Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
-      File dnRoot = it.next();
+    for(StorageLocation dir : dataDirs) {
+      File dnRoot = dir.getFile();
       File bpRoot = BlockPoolSliceStorage.getBpRoot(bpID, new File(dnRoot,
           STORAGE_DIR_CURRENT));
       bpDataDirs.add(bpRoot);
@@ -263,19 +258,28 @@ public class DataStorage extends Storage {
     }
   }
 
-  void format(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
+  void format(StorageDirectory sd, NamespaceInfo nsInfo,
+              String datanodeUuid) throws IOException {
     sd.clearDirectory(); // create directory
     this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
     this.clusterID = nsInfo.getClusterID();
     this.namespaceID = nsInfo.getNamespaceID();
     this.cTime = 0;
-    // store storageID as it currently is
+    this.datanodeUuid = datanodeUuid;
+
+    if (sd.getStorageUuid() == null) {
+      // Assign a new Storage UUID.
+      sd.setStorageUuid(DatanodeStorage.generateUuid());
+    }
+
     writeProperties(sd);
   }
 
   /*
    * Set ClusterID, StorageID, StorageType, CTime into
-   * DataStorage VERSION file
+   * DataStorage VERSION file.
+   * Always called just before writing the properties to
+   * the VERSION file.
   */
   @Override
   protected void setPropertiesFromFields(Properties props, 
@@ -285,7 +289,13 @@ public class DataStorage extends Storage {
     props.setProperty("clusterID", clusterID);
     props.setProperty("cTime", String.valueOf(cTime));
     props.setProperty("layoutVersion", String.valueOf(layoutVersion));
-    props.setProperty("storageID", getStorageID());
+    props.setProperty("storageID", sd.getStorageUuid());
+
+    String datanodeUuid = getDatanodeUuid();
+    if (datanodeUuid != null) {
+      props.setProperty("datanodeUuid", datanodeUuid);
+    }
+
     // Set NamespaceID in version before federation
     if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
       props.setProperty("namespaceID", String.valueOf(namespaceID));
@@ -295,6 +305,7 @@ public class DataStorage extends Storage {
   /*
    * Read ClusterID, StorageID, StorageType, CTime from 
    * DataStorage VERSION file and verify them.
+   * Always called just after reading the properties from the VERSION file.
    */
   @Override
   protected void setFieldsFromProperties(Properties props, StorageDirectory sd)
@@ -318,20 +329,36 @@ public class DataStorage extends Storage {
       setNamespaceID(props, sd);
     }
     
+
     // valid storage id, storage id may be empty
     String ssid = props.getProperty("storageID");
     if (ssid == null) {
       throw new InconsistentFSStateException(sd.getRoot(), "file "
           + STORAGE_FILE_VERSION + " is invalid.");
     }
-    String sid = getStorageID();
-    if (!(sid.equals("") || ssid.equals("") || sid.equals(ssid))) {
+    String sid = sd.getStorageUuid();
+    if (!(sid == null || sid.equals("") ||
+          ssid.equals("") || sid.equals(ssid))) {
       throw new InconsistentFSStateException(sd.getRoot(),
           "has incompatible storage Id.");
     }
-    
-    if (sid.equals("")) { // update id only if it was empty
-      setStorageID(ssid);
+
+    if (sid == null) { // update id only if it was null
+      sd.setStorageUuid(ssid);
+    }
+
+    // Update the datanode UUID if present.
+    if (props.getProperty("datanodeUuid") != null) {
+      String dnUuid = props.getProperty("datanodeUuid");
+
+      if (getDatanodeUuid() == null) {
+        setDatanodeUuid(dnUuid);
+      } else if (getDatanodeUuid().compareTo(dnUuid) != 0) {
+        throw new InconsistentFSStateException(sd.getRoot(),
+            "Root " + sd.getRoot() + ": DatanodeUuid=" + dnUuid +
+            ", does not match " + getDatanodeUuid() + " from other" +
+            " StorageDirectory.");
+      }
     }
   }
 

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

@@ -284,7 +284,7 @@ class DataXceiver extends Receiver implements Runnable {
         BlockSender.ClientTraceLog.info(String.format(
             "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
             " blockid: %s, srvID: %s, success: %b",
-            blk.getBlockId(), dnR.getStorageID(), (fis != null)
+            blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null)
           ));
       }
       if (fis != null) {
@@ -317,7 +317,7 @@ class DataXceiver extends Receiver implements Runnable {
       clientName.length() > 0 && ClientTraceLog.isInfoEnabled()
         ? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress,
             "%d", "HDFS_READ", clientName, "%d",
-            dnR.getStorageID(), block, "%d")
+            dnR.getDatanodeUuid(), block, "%d")
         : dnR + " Served block " + block + " to " +
             remoteAddress;
 
@@ -447,6 +447,7 @@ class DataXceiver extends Receiver implements Runnable {
     String mirrorNode = null;           // the name:port of next target
     String firstBadLink = "";           // first datanode that failed in connection setup
     Status mirrorInStatus = SUCCESS;
+    final String storageUuid;
     try {
       if (isDatanode || 
           stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
@@ -457,8 +458,10 @@ class DataXceiver extends Receiver implements Runnable {
             stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
             clientname, srcDataNode, datanode, requestedChecksum,
             cachingStrategy);
+        storageUuid = blockReceiver.getStorageUuid();
       } else {
-        datanode.data.recoverClose(block, latestGenerationStamp, minBytesRcvd);
+        storageUuid = datanode.data.recoverClose(
+            block, latestGenerationStamp, minBytesRcvd);
       }
 
       //
@@ -590,7 +593,7 @@ class DataXceiver extends Receiver implements Runnable {
       // the block is finalized in the PacketResponder.
       if (isDatanode ||
           stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
-        datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
+        datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT, storageUuid);
         LOG.info("Received " + block + " src: " + remoteAddress + " dest: "
             + localAddress + " of size " + block.getNumBytes());
       }
@@ -859,9 +862,11 @@ class DataXceiver extends Receiver implements Runnable {
           dataXceiverServer.balanceThrottler, null);
                     
       // notify name node
-      datanode.notifyNamenodeReceivedBlock(block, delHint);
+      datanode.notifyNamenodeReceivedBlock(
+          block, delHint, blockReceiver.getStorageUuid());
 
-      LOG.info("Moved " + block + " from " + peer.getRemoteAddressString());
+      LOG.info("Moved " + block + " from " + peer.getRemoteAddressString()
+          + ", delHint=" + delHint);
       
     } catch (IOException ioe) {
       opStatus = ERROR;

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

@@ -77,18 +77,6 @@ public class DatanodeJspHelper {
       });
   }
 
-  /**
-   * Internal convenience method for canonicalizing host name.
-   * @param addr name:port or name 
-   * @return canonicalized host name
-   */
-  private static String canonicalize(String addr) {
-    // default port 1 is supplied to allow addr without port.
-    // the port will be ignored.
-    return NetUtils.createSocketAddr(addr, 1).getAddress()
-           .getCanonicalHostName();
-  }
-
   /**
    * Get the default chunk size.
    * @param conf the configuration
@@ -228,7 +216,7 @@ public class DatanodeJspHelper {
       }
     }
     out.print("<br><a href=\"///"
-        + canonicalize(nnAddr) + ":"
+        + JspHelper.canonicalize(nnAddr) + ":"
         + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
     dfs.close();
   }
@@ -359,7 +347,7 @@ public class DatanodeJspHelper {
     // generate a table and dump the info
     out.println("\n<table>");
     
-    String nnCanonicalName = canonicalize(nnAddr);
+    String nnCanonicalName = JspHelper.canonicalize(nnAddr);
     for (LocatedBlock cur : blocks) {
       out.print("<tr>");
       final String blockidstring = Long.toString(cur.getBlock().getBlockId());

+ 10 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -230,10 +229,6 @@ public class DirectoryScanner implements Runnable {
       throw new RuntimeException(prefix + " is not a prefix of " + fullPath);
     }
 
-    ScanInfo(long blockId) {
-      this(blockId, null, null, null);
-    }
-
     ScanInfo(long blockId, File blockFile, File metaFile, FsVolumeSpi vol) {
       this.blockId = blockId;
       String condensedVolPath = vol == null ? null :
@@ -439,8 +434,8 @@ public class DirectoryScanner implements Runnable {
         diffs.put(bpid, diffRecord);
         
         statsRecord.totalBlocks = blockpoolReport.length;
-        List<Block> bl = dataset.getFinalizedBlocks(bpid);
-        Block[] memReport = bl.toArray(new Block[bl.size()]);
+        List<FinalizedReplica> bl = dataset.getFinalizedBlocks(bpid);
+        FinalizedReplica[] memReport = bl.toArray(new FinalizedReplica[bl.size()]);
         Arrays.sort(memReport); // Sort based on blockId
   
         int d = 0; // index for blockpoolReport
@@ -458,7 +453,8 @@ public class DirectoryScanner implements Runnable {
           }
           if (info.getBlockId() > memBlock.getBlockId()) {
             // Block is missing on the disk
-            addDifference(diffRecord, statsRecord, memBlock.getBlockId());
+            addDifference(diffRecord, statsRecord,
+                          memBlock.getBlockId(), info.getVolume());
             m++;
             continue;
           }
@@ -478,7 +474,9 @@ public class DirectoryScanner implements Runnable {
           m++;
         }
         while (m < memReport.length) {
-          addDifference(diffRecord, statsRecord, memReport[m++].getBlockId());
+          FinalizedReplica current = memReport[m++];
+          addDifference(diffRecord, statsRecord,
+                        current.getBlockId(), current.getVolume());
         }
         while (d < blockpoolReport.length) {
           statsRecord.missingMemoryBlocks++;
@@ -502,10 +500,11 @@ public class DirectoryScanner implements Runnable {
 
   /** Block is not found on the disk */
   private void addDifference(LinkedList<ScanInfo> diffRecord,
-                             Stats statsRecord, long blockId) {
+                             Stats statsRecord, long blockId,
+                             FsVolumeSpi vol) {
     statsRecord.missingBlockFile++;
     statsRecord.missingMetaFile++;
-    diffRecord.add(new ScanInfo(blockId));
+    diffRecord.add(new ScanInfo(blockId, null, null, vol));
   }
 
   /** Is the given volume still valid in the dataset? */

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/Replica.java

@@ -54,4 +54,9 @@ public interface Replica {
    * @return the number of bytes that are visible to readers
    */
   public long getVisibleLength();
+
+  /**
+   * Return the storageUuid of the volume that stores this replica.
+   */
+  public String getStorageUuid();
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -137,6 +137,14 @@ abstract public class ReplicaInfo extends Block implements Replica {
   void setVolume(FsVolumeSpi vol) {
     this.volume = vol;
   }
+
+  /**
+   * Get the storageUuid of the volume that stores this replica.
+   */
+  @Override
+  public String getStorageUuid() {
+    return volume.getStorageID();
+  }
   
   /**
    * Return the parent directory path where this replica is located

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java

@@ -87,6 +87,7 @@ public class SecureDataNodeStarter implements Daemon {
   public static SecureResources getSecureResources(Configuration conf)
       throws Exception {
     HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
+    boolean isSecure = UserGroupInformation.isSecurityEnabled();
 
     // Obtain secure port for data streaming to datanode
     InetSocketAddress streamingAddr  = DataNode.getStreamingAddr(conf);
@@ -106,6 +107,11 @@ public class SecureDataNodeStarter implements Daemon {
               + ss.getLocalPort());
     }
 
+    if (ss.getLocalPort() > 1023 && isSecure) {
+      throw new RuntimeException(
+        "Cannot start secure datanode with unprivileged RPC ports");
+    }
+
     System.err.println("Opened streaming server at " + streamingAddr);
 
     // Bind a port for the web server. The code intends to bind HTTP server to
@@ -126,9 +132,9 @@ public class SecureDataNodeStarter implements Daemon {
       System.err.println("Successfully obtained privileged resources (streaming port = "
           + ss + " ) (http listener port = " + listener.getConnection() +")");
 
-      if ((ss.getLocalPort() > 1023 || listener.getPort() > 1023) &&
-          UserGroupInformation.isSecurityEnabled()) {
-        throw new RuntimeException("Cannot start secure datanode with unprivileged ports");
+      if (listener.getPort() > 1023 && isSecure) {
+        throw new RuntimeException(
+            "Cannot start secure datanode with unprivileged HTTP ports");
       }
       System.err.println("Opened info server at " + infoSocAddr);
     }

+ 101 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java

@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.util.regex.Pattern;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.regex.Matcher;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.server.common.Util;
+
+/**
+ * Encapsulates the URI and storage medium that together describe a
+ * storage directory.
+ * The default storage medium is assumed to be DISK, if none is specified.
+ *
+ */
+@InterfaceAudience.Private
+public class StorageLocation {
+  final StorageType storageType;
+  final File file;
+
+  /** Regular expression that describes a storage uri with a storage type.
+   *  e.g. [Disk]/storages/storage1/
+   */
+  private static final Pattern regex = Pattern.compile("^\\[(\\w*)\\](.+)$");
+
+  private StorageLocation(StorageType storageType, URI uri) {
+    this.storageType = storageType;
+
+    if (uri.getScheme() == null ||
+        "file".equalsIgnoreCase(uri.getScheme())) {
+      // drop any (illegal) authority in the URI for backwards compatibility
+      this.file = new File(uri.getPath());
+    } else {
+      throw new IllegalArgumentException("Unsupported URI schema in " + uri);
+    }
+  }
+
+  public StorageType getStorageType() {
+    return this.storageType;
+  }
+
+  URI getUri() {
+    return file.toURI();
+  }
+
+  public File getFile() {
+    return this.file;
+  }
+
+  /**
+   * Attempt to parse a storage uri with storage class and URI. The storage
+   * class component of the uri is case-insensitive.
+   *
+   * @param rawLocation Location string of the format [type]uri, where [type] is
+   *                    optional.
+   * @return A StorageLocation object if successfully parsed, null otherwise.
+   *         Does not throw any exceptions.
+   */
+  static StorageLocation parse(String rawLocation) throws IOException {
+    Matcher matcher = regex.matcher(rawLocation);
+    StorageType storageType = StorageType.DEFAULT;
+    String location = rawLocation;
+
+    if (matcher.matches()) {
+      String classString = matcher.group(1);
+      location = matcher.group(2);
+      if (!classString.isEmpty()) {
+        storageType = StorageType.valueOf(classString.toUpperCase());
+      }
+    }
+
+    return new StorageLocation(storageType, Util.stringAsURI(location));
+  }
+
+  @Override
+  public String toString() {
+    return "[" + storageType + "]" + file.toURI();
+  }
+}

部分文件因文件數量過多而無法顯示