浏览代码

Merge trunk to HDFS-4685.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1552467 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 11 年之前
父节点
当前提交
72b0b1881f
共有 100 个文件被更改,包括 5350 次插入1655 次删除
  1. 21 5
      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. 16 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
  8. 22 22
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  9. 9 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
  10. 51 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java
  11. 51 36
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  12. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
  13. 17 0
      hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory
  14. 112 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProvider.java
  15. 191 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
  16. 1 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  17. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
  18. 19 4
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
  19. 47 15
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  20. 1 8
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  21. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/CacheFlag.java
  22. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
  23. 18 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
  24. 530 393
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  25. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
  26. 15 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  27. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  28. 43 28
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  29. 12 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  30. 25 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  31. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
  32. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  33. 12 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  34. 10 82
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
  35. 17 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
  36. 9 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  37. 7 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  38. 19 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  39. 26 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  40. 10 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  41. 186 41
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  42. 15 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  43. 26 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  44. 24 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/CachingStrategy.java
  45. 2 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  46. 199 86
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  47. 25 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
  48. 4 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  49. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  50. 33 186
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  51. 13 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
  52. 205 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  53. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  54. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
  55. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  56. 19 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  57. 66 49
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
  58. 10 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  59. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  60. 18 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  61. 310 46
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
  62. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java
  63. 81 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  64. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
  65. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
  66. 342 305
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  67. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  68. 30 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  69. 9 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
  70. 二进制
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  71. 92 94
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  72. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml
  73. 12 0
      hadoop-mapreduce-project/CHANGES.txt
  74. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  75. 20 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  76. 47 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  77. 117 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/preemption/AMPreemptionPolicy.java
  78. 111 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/preemption/KillAMPreemptionPolicy.java
  79. 72 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/preemption/NoopAMPreemptionPolicy.java
  80. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
  81. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  82. 8 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
  83. 4 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
  84. 11 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
  85. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobCounter.java
  86. 10 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  87. 30 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/CheckpointID.java
  88. 31 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/CheckpointNamingService.java
  89. 100 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/CheckpointService.java
  90. 26 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/EnumCounter.java
  91. 72 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/FSCheckpointID.java
  92. 193 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/FSCheckpointService.java
  93. 32 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/RandomNameCNS.java
  94. 39 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/SimpleNamingService.java
  95. 126 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/TaskCheckpointID.java
  96. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/org/apache/hadoop/mapreduce/JobCounter.properties
  97. 48 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/checkpoint/TestFSCheckpointID.java
  98. 102 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/checkpoint/TestFSCheckpointService.java
  99. 39 39
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java
  100. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java

+ 21 - 5
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.
@@ -282,9 +285,6 @@ Trunk (Unreleased)
 
     HADOOP-10044 Improve the javadoc of rpc code (sanjay Radia)
 
-    HADOOP-8753. LocalDirAllocator throws "ArithmeticException: / by zero" when
-    there is no available space on configured local dir. (Benoy Antony via hitesh)
-
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -400,12 +400,18 @@ 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)
+
   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)
 
   BUG FIXES
 
@@ -470,6 +476,16 @@ 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)
+
 Release 2.3.0 - UNRELEASED
 
   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;
+    }
+  }
+}

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

@@ -68,7 +68,14 @@ public class RetryPolicies {
    * </p>
    */
   public static final RetryPolicy RETRY_FOREVER = new RetryForever();
-  
+
+  /**
+   * <p>
+   * Keep failing over forever
+   * </p>
+   */
+  public static final RetryPolicy FAILOVER_FOREVER = new FailoverForever();
+
   /**
    * <p>
    * Keep trying a limited number of times, waiting a fixed time between attempts,
@@ -166,6 +173,14 @@ public class RetryPolicies {
       return RetryAction.RETRY;
     }
   }
+
+  static class FailoverForever implements RetryPolicy {
+    @Override
+    public RetryAction shouldRetry(Exception e, int retries, int failovers,
+        boolean isIdempotentOrAtMostOnce) throws Exception {
+      return RetryAction.FAILOVER_AND_RETRY;
+    }
+  }
   
   /**
    * Retry up to maxRetries.

+ 22 - 22
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);
         }
@@ -2011,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);
@@ -2019,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;
@@ -2052,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
@@ -2101,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");
     }
 
   }

+ 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

+ 51 - 1
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"),
@@ -151,7 +157,7 @@ public class SaslRpcServer {
         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);
           }
         });
@@ -180,6 +186,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 +370,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")

+ 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());
+  }
+}

+ 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;

+ 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());
   }
-}
+}

+ 47 - 15
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -216,36 +216,21 @@ 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)
 
@@ -253,6 +238,9 @@ Trunk (Unreleased)
     INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
     (jing9 via szetszwo)
 
+    HDFS-5431. Support cachepool-based limit management in path-based caching
+    (awang via cmccabe)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
@@ -460,6 +448,9 @@ Trunk (Unreleased)
     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)
+
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
 
     HDFS-4985. Add storage type to the protocol and expose it in block report
@@ -741,6 +732,37 @@ Release 2.4.0 - UNRELEASED
     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)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -796,6 +818,8 @@ 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)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -826,6 +850,9 @@ Release 2.3.0 - UNRELEASED
     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)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -970,6 +997,11 @@ Release 2.3.0 - UNRELEASED
     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)
 
 Release 2.2.0 - 2013-10-13
 

+ 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>

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/CacheFlag.java

@@ -0,0 +1,44 @@
+/**
+ * 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.fs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Specifies semantics for CacheDirective operations. Multiple flags can
+ * be combined in an EnumSet.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+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;
+  }
+
+  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.AclEntry;
@@ -111,6 +112,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;
@@ -123,7 +125,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;
@@ -284,6 +285,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;
@@ -357,6 +360,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,
@@ -2297,20 +2306,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();
     }

+ 4 - 0
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";

+ 43 - 28
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;
@@ -227,7 +228,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         dfsClient.getConf().shortCircuitStreamsCacheSize,
         dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
     this.cachingStrategy =
-        dfsClient.getDefaultReadCachingStrategy().duplicate();
+        dfsClient.getDefaultReadCachingStrategy();
     openInfo();
   }
 
@@ -236,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
@@ -246,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;
@@ -573,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);
@@ -927,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;
@@ -939,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(): " +
@@ -1052,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,
@@ -1063,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.
@@ -1073,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
@@ -1109,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. " +
@@ -1132,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);
@@ -1156,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 " +
@@ -1176,7 +1192,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         dfsClient.getConf(), file, block, blockToken, startOffset,
         len, verifyChecksum, clientName, peer, chosenNode, 
         dsFactory, peerCache, fileInputStreamCache, false,
-        cachingStrategy);
+        curCachingStrategy);
   }
 
 
@@ -1450,14 +1466,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();
   }
 
@@ -1479,23 +1499,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.
@@ -1518,8 +1534,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 " +

+ 12 - 5
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 {
@@ -1183,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(
@@ -1378,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);
@@ -1993,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

+ 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;
@@ -1587,40 +1588,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;
   }
 }

+ 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);
   }
 
   /**

+ 10 - 82
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,6 @@ 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;
 
 /**
  * CachePoolInfo describes a cache pool.
@@ -64,7 +54,7 @@ public class CachePoolInfo {
   FsPermission mode;
 
   @Nullable
-  Integer weight;
+  Long limit;
 
   public CachePoolInfo(String poolName) {
     this.poolName = poolName;
@@ -101,12 +91,12 @@ public class CachePoolInfo {
     return this;
   }
 
-  public Integer getWeight() {
-    return weight;
+  public Long getLimit() {
+    return limit;
   }
 
-  public CachePoolInfo setWeight(Integer weight) {
-    this.weight = weight;
+  public CachePoolInfo setLimit(Long bytes) {
+    this.limit = bytes;
     return this;
   }
 
@@ -117,7 +107,7 @@ 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("}").toString();
   }
   
@@ -134,7 +124,7 @@ public class CachePoolInfo {
         append(ownerName, other.ownerName).
         append(groupName, other.groupName).
         append(mode, other.mode).
-        append(weight, other.weight).
+        append(limit, other.limit).
         isEquals();
   }
 
@@ -145,7 +135,7 @@ public class CachePoolInfo {
         append(ownerName).
         append(groupName).
         append(mode).
-        append(weight).
+        append(limit).
         hashCode();
   }
 
@@ -153,8 +143,8 @@ 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.");
     }
     validateName(info.poolName);
   }
@@ -167,66 +157,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();

+ 9 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -19,10 +19,12 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.List;
 
 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;
@@ -1103,23 +1105,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.

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -350,7 +350,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) {
@@ -1064,9 +1064,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);
     }
@@ -1078,7 +1080,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);

+ 19 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -21,11 +21,13 @@ import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.List;
 
 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;
@@ -1012,24 +1014,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);
     }

+ 26 - 6
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;
@@ -75,6 +76,7 @@ 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;
@@ -1205,7 +1207,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) {
@@ -1220,7 +1222,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;
@@ -1818,8 +1836,8 @@ 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());
     }
     return builder.build();
   }
@@ -1837,8 +1855,8 @@ 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());
     }
     return info;
   }
@@ -1847,6 +1865,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();
@@ -1856,6 +1875,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();

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

@@ -2893,6 +2893,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    */
   boolean isReplicationInProgress(DatanodeDescriptor srcNode) {
     boolean status = false;
+    boolean firstReplicationLog = true;
     int underReplicatedBlocks = 0;
     int decommissionOnlyReplicas = 0;
     int underReplicatedInOpenFiles = 0;
@@ -2907,10 +2908,17 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         int curExpectedReplicas = getReplication(block);
         if (isNeededReplication(block, curExpectedReplicas, curReplicas)) {
           if (curExpectedReplicas > curReplicas) {
-            //Log info about one block for this node which needs replication
+            // Log info about one block for this node which needs replication
             if (!status) {
               status = true;
-              logBlockReplicationInfo(block, srcNode, num);
+              if (firstReplicationLog) {
+                logBlockReplicationInfo(block, srcNode, num);
+              }
+              // Allowing decommission as long as default replication is met
+              if (curReplicas >= defaultReplication) {
+                status = false;
+                firstReplicationLog = false;
+              }
             }
             underReplicatedBlocks++;
             if ((curReplicas == 0) && (num.decommissionedReplicas() > 0)) {

+ 186 - 41
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,52 @@ 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 = new ReentrantLock();
 
   /**
-   * 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 = lock.newCondition();
 
   /**
-   * Mark status of the current scan.
+   * Notifies waiting threads that a rescan has finished.
    */
-  private boolean mark = false;
+  private final Condition scanFinished = lock.newCondition();
+
+  /**
+   * 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. Protected by the CRM lock.
+   */
+  private boolean shutdown = false;
 
   /**
-   * True if this monitor should terminate.
+   * The monotonic time at which the current scan started.
+   */
+  private long startTimeMs;
+
+  /**
+   * Mark status of the current scan.
    */
-  private boolean shutdown;
+  private boolean mark = false;
 
   /**
    * Cache directives found in the previous scan.
@@ -108,7 +140,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
    * Blocks found in the previous scan.
    */
   private long scannedBlocks;
-  
+
   public CacheReplicationMonitor(FSNamesystem namesystem,
       CacheManager cacheManager, long intervalMs) {
     this.namesystem = namesystem;
@@ -120,41 +152,60 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
 
   @Override
   public void run() {
-    shutdown = false;
-    rescanImmediately = true;
-    scanTimeMs = 0;
+    startTimeMs = 0;
     LOG.info("Starting CacheReplicationMonitor with interval " +
              intervalMs + " milliseconds");
     try {
       long curTimeMs = Time.monotonicNow();
       while (true) {
-        synchronized(this) {
+        // Not all of the variables accessed here need the CRM lock, but take
+        // it anyway for simplicity
+        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();
           }
+        } finally {
+          lock.unlock();
         }
-        scanTimeMs = curTimeMs;
+        // Mark scan as started, clear needsRescan
+        lock.lock();
+        try {
+          isScanning = true;
+          needsRescan = false;
+        } finally {
+          lock.unlock();
+        }
+        startTimeMs = curTimeMs;
         mark = !mark;
         rescan();
         curTimeMs = Time.monotonicNow();
+        // Retake the CRM lock to 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 (Throwable t) {
@@ -164,15 +215,91 @@ 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.
+   * Similar to {@link CacheReplicationMonitor#waitForRescan()}, except it only
+   * waits if there are pending operations that necessitate a rescan as
+   * indicated by {@link #setNeedsRescan()}.
+   * <p>
+   * Note that this call may release the FSN lock, so operations before and
+   * after are not necessarily atomic.
+   */
+  public void waitForRescanIfNeeded() {
+    lock.lock();
+    try {
+      if (!needsRescan) {
+        return;
+      }
+    } finally {
+      lock.unlock();
+    }
+    waitForRescan();
+  }
+
+  /**
+   * 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 void waitForRescan() {
+    // Drop the FSN lock temporarily and retake it after we finish waiting
+    // Need to handle both the read lock and the write lock
+    boolean retakeWriteLock = false;
+    if (namesystem.hasWriteLock()) {
+      namesystem.writeUnlock();
+      retakeWriteLock = true;
+    } else if (namesystem.hasReadLock()) {
+      namesystem.readUnlock();
+    } else {
+      // Expected to have at least one of the locks
+      Preconditions.checkState(false,
+          "Need to be holding either the read or write lock");
+    }
+    // try/finally for retaking FSN lock
+    try {
+      lock.lock();
+      // try/finally for releasing CRM lock
+      try {
+        // If no scan is already ongoing, mark the CRM as dirty and kick
+        if (!isScanning) {
+          needsRescan = true;
+          doRescan.signal();
+        }
+        // Wait until the scan finishes and the count advances
+        final long startCount = scanCount;
+        while (startCount >= scanCount) {
+          try {
+            scanFinished.await();
+          } catch (InterruptedException e) {
+            LOG.warn("Interrupted while waiting for CacheReplicationMonitor"
+                + " rescan", e);
+            break;
+          }
+        }
+      } finally {
+        lock.unlock();
+      }
+    } finally {
+      if (retakeWriteLock) {
+        namesystem.writeLock();
+      } else {
+        namesystem.readLock();
+      }
+    }
+  }
+
+  /**
+   * Indicates to the CacheReplicationMonitor that there have been CacheManager
+   * changes that require a rescan.
    */
-  public synchronized void kick() {
-    rescanImmediately = true;
-    this.notifyAll();
+  public void setNeedsRescan() {
+    lock.lock();
+    try {
+      this.needsRescan = true;
+    } finally {
+      lock.unlock();
+    }
   }
 
   /**
@@ -180,10 +307,14 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
    */
   @Override
   public void close() throws IOException {
-    synchronized(this) {
+    lock.lock();
+    try {
       if (shutdown) return;
       shutdown = true;
-      this.notifyAll();
+      doRescan.signalAll();
+      scanFinished.signalAll();
+    } finally {
+      lock.unlock();
     }
     try {
       if (this.isAlive()) {
@@ -228,12 +359,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);
+              + now + ")");
         }
         continue;
       }
@@ -280,15 +413,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) {

+ 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);
       }

+ 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 

+ 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 + ")";

+ 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());

+ 199 - 86
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -27,11 +27,12 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT;
 
 import java.io.DataInput;
-import java.io.DataOutput;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
@@ -45,13 +46,16 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirective;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -68,7 +72,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -341,6 +345,67 @@ public final class CacheManager {
     return expiryTime;
   }
 
+  /**
+   * Throws an exception if the CachePool does not have enough capacity to
+   * cache the given path at the replication factor.
+   *
+   * @param pool CachePool where the path is being cached
+   * @param path Path that is being cached
+   * @param replication Replication factor of the path
+   * @throws InvalidRequestException if the pool does not have enough capacity
+   */
+  private void checkLimit(CachePool pool, String path,
+      short replication) throws InvalidRequestException {
+    CacheDirectiveStats stats = computeNeeded(path, replication);
+    if (pool.getBytesNeeded() + (stats.getBytesNeeded() * replication) > pool
+        .getLimit()) {
+      throw new InvalidRequestException("Caching path " + path + " of size "
+          + stats.getBytesNeeded() / replication + " bytes at replication "
+          + replication + " would exceed pool " + pool.getPoolName()
+          + "'s remaining capacity of "
+          + (pool.getLimit() - pool.getBytesNeeded()) + " bytes.");
+    }
+  }
+
+  /**
+   * Computes the needed number of bytes and files for a path.
+   * @return CacheDirectiveStats describing the needed stats for this path
+   */
+  private CacheDirectiveStats computeNeeded(String path, short replication) {
+    FSDirectory fsDir = namesystem.getFSDirectory();
+    INode node;
+    long requestedBytes = 0;
+    long requestedFiles = 0;
+    CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
+    try {
+      node = fsDir.getINode(path);
+    } catch (UnresolvedLinkException e) {
+      // We don't cache through symlinks
+      return builder.build();
+    }
+    if (node == null) {
+      return builder.build();
+    }
+    if (node.isFile()) {
+      requestedFiles = 1;
+      INodeFile file = node.asFile();
+      requestedBytes = file.computeFileSize();
+    } else if (node.isDirectory()) {
+      INodeDirectory dir = node.asDirectory();
+      ReadOnlyList<INode> children = dir.getChildrenList(null);
+      requestedFiles = children.size();
+      for (INode child : children) {
+        if (child.isFile()) {
+          requestedBytes += child.asFile().computeFileSize();
+        }
+      }
+    }
+    return new CacheDirectiveStats.Builder()
+        .setBytesNeeded(requestedBytes)
+        .setFilesCached(requestedFiles)
+        .build();
+  }
+
   /**
    * Get a CacheDirective by ID, validating the ID and that the directive
    * exists.
@@ -384,6 +449,15 @@ public final class CacheManager {
       directivesByPath.put(path, directives);
     }
     directives.add(directive);
+    // Fix up pool stats
+    CacheDirectiveStats stats =
+        computeNeeded(directive.getPath(), directive.getReplication());
+    directive.addBytesNeeded(stats.getBytesNeeded());
+    directive.addFilesNeeded(directive.getFilesNeeded());
+
+    if (monitor != null) {
+      monitor.setNeedsRescan();
+    }
   }
 
   /**
@@ -407,7 +481,7 @@ public final class CacheManager {
   }
 
   public CacheDirectiveInfo addDirective(
-      CacheDirectiveInfo info, FSPermissionChecker pc)
+      CacheDirectiveInfo info, FSPermissionChecker pc, EnumSet<CacheFlag> flags)
       throws IOException {
     assert namesystem.hasWriteLock();
     CacheDirective directive;
@@ -418,6 +492,14 @@ public final class CacheManager {
       short replication = validateReplication(info, (short)1);
       long expiryTime = validateExpiryTime(info,
           CacheDirectiveInfo.Expiration.EXPIRY_NEVER);
+      // Do quota validation if required
+      if (!flags.contains(CacheFlag.FORCE)) {
+        // Can't kick and wait if caching is disabled
+        if (monitor != null) {
+          monitor.waitForRescan();
+        }
+        checkLimit(pool, path, replication);
+      }
       // All validation passed
       // Add a new entry with the next available ID.
       long id = getNextDirectiveId();
@@ -428,14 +510,11 @@ public final class CacheManager {
       throw e;
     }
     LOG.info("addDirective of " + info + " successful.");
-    if (monitor != null) {
-      monitor.kick();
-    }
     return directive.toInfo();
   }
 
   public void modifyDirective(CacheDirectiveInfo info,
-      FSPermissionChecker pc) throws IOException {
+      FSPermissionChecker pc, EnumSet<CacheFlag> flags) throws IOException {
     assert namesystem.hasWriteLock();
     String idString =
         (info.getId() == null) ?
@@ -463,6 +542,13 @@ public final class CacheManager {
       if (info.getPool() != null) {
         pool = getCachePool(validatePoolName(info));
         checkWritePermission(pc, pool);
+        if (!flags.contains(CacheFlag.FORCE)) {
+          // Can't kick and wait if caching is disabled
+          if (monitor != null) {
+            monitor.waitForRescan();
+          }
+          checkLimit(pool, path, replication);
+        }
       }
       removeInternal(prevEntry);
       CacheDirective newEntry =
@@ -489,9 +575,18 @@ public final class CacheManager {
     if (directives.size() == 0) {
       directivesByPath.remove(path);
     }
+    // Fix up the stats from removing the pool
+    final CachePool pool = directive.getPool();
+    directive.addBytesNeeded(-directive.getBytesNeeded());
+    directive.addFilesNeeded(-directive.getFilesNeeded());
+
     directivesById.remove(directive.getId());
-    directive.getPool().getDirectiveList().remove(directive);
+    pool.getDirectiveList().remove(directive);
     assert directive.getPool() == null;
+
+    if (monitor != null) {
+      monitor.setNeedsRescan();
+    }
   }
 
   public void removeDirective(long id, FSPermissionChecker pc)
@@ -505,9 +600,6 @@ public final class CacheManager {
       LOG.warn("removeDirective of " + id + " failed: ", e);
       throw e;
     }
-    if (monitor != null) {
-      monitor.kick();
-    }
     LOG.info("removeDirective of " + id + " successful.");
   }
 
@@ -527,6 +619,9 @@ public final class CacheManager {
     if (filter.getReplication() != null) {
       throw new IOException("Filtering by replication is unsupported.");
     }
+    if (monitor != null) {
+      monitor.waitForRescanIfNeeded();
+    }
     ArrayList<CacheDirectiveEntry> replies =
         new ArrayList<CacheDirectiveEntry>(NUM_PRE_ALLOCATED_ENTRIES);
     int numReplies = 0;
@@ -573,16 +668,22 @@ public final class CacheManager {
   public CachePoolInfo addCachePool(CachePoolInfo info)
       throws IOException {
     assert namesystem.hasWriteLock();
-    CachePoolInfo.validate(info);
-    String poolName = info.getPoolName();
-    CachePool pool = cachePools.get(poolName);
-    if (pool != null) {
-      throw new InvalidRequestException("Cache pool " + poolName
-          + " already exists.");
+    CachePool pool;
+    try {
+      CachePoolInfo.validate(info);
+      String poolName = info.getPoolName();
+      pool = cachePools.get(poolName);
+      if (pool != null) {
+        throw new InvalidRequestException("Cache pool " + poolName
+            + " already exists.");
+      }
+      pool = CachePool.createFromInfoAndDefaults(info);
+      cachePools.put(pool.getPoolName(), pool);
+    } catch (IOException e) {
+      LOG.info("addCachePool of " + info + " failed: ", e);
+      throw e;
     }
-    pool = CachePool.createFromInfoAndDefaults(info);
-    cachePools.put(pool.getPoolName(), pool);
-    LOG.info("Created new cache pool " + pool);
+    LOG.info("addCachePool of " + info + " successful.");
     return pool.getInfo(true);
   }
 
@@ -597,42 +698,51 @@ public final class CacheManager {
   public void modifyCachePool(CachePoolInfo info)
       throws IOException {
     assert namesystem.hasWriteLock();
-    CachePoolInfo.validate(info);
-    String poolName = info.getPoolName();
-    CachePool pool = cachePools.get(poolName);
-    if (pool == null) {
-      throw new InvalidRequestException("Cache pool " + poolName
-          + " does not exist.");
-    }
     StringBuilder bld = new StringBuilder();
-    String prefix = "";
-    if (info.getOwnerName() != null) {
-      pool.setOwnerName(info.getOwnerName());
-      bld.append(prefix).
-        append("set owner to ").append(info.getOwnerName());
-      prefix = "; ";
-    }
-    if (info.getGroupName() != null) {
-      pool.setGroupName(info.getGroupName());
-      bld.append(prefix).
-        append("set group to ").append(info.getGroupName());
-      prefix = "; ";
-    }
-    if (info.getMode() != null) {
-      pool.setMode(info.getMode());
-      bld.append(prefix).append("set mode to " + info.getMode());
-      prefix = "; ";
-    }
-    if (info.getWeight() != null) {
-      pool.setWeight(info.getWeight());
-      bld.append(prefix).
-        append("set weight to ").append(info.getWeight());
-      prefix = "; ";
-    }
-    if (prefix.isEmpty()) {
-      bld.append("no changes.");
+    try {
+      CachePoolInfo.validate(info);
+      String poolName = info.getPoolName();
+      CachePool pool = cachePools.get(poolName);
+      if (pool == null) {
+        throw new InvalidRequestException("Cache pool " + poolName
+            + " does not exist.");
+      }
+      String prefix = "";
+      if (info.getOwnerName() != null) {
+        pool.setOwnerName(info.getOwnerName());
+        bld.append(prefix).
+          append("set owner to ").append(info.getOwnerName());
+        prefix = "; ";
+      }
+      if (info.getGroupName() != null) {
+        pool.setGroupName(info.getGroupName());
+        bld.append(prefix).
+          append("set group to ").append(info.getGroupName());
+        prefix = "; ";
+      }
+      if (info.getMode() != null) {
+        pool.setMode(info.getMode());
+        bld.append(prefix).append("set mode to " + info.getMode());
+        prefix = "; ";
+      }
+      if (info.getLimit() != null) {
+        pool.setLimit(info.getLimit());
+        bld.append(prefix).append("set limit to " + info.getLimit());
+        prefix = "; ";
+        // New limit changes stats, need to set needs refresh
+        if (monitor != null) {
+          monitor.setNeedsRescan();
+        }
+      }
+      if (prefix.isEmpty()) {
+        bld.append("no changes.");
+      }
+    } catch (IOException e) {
+      LOG.info("modifyCachePool of " + info + " failed: ", e);
+      throw e;
     }
-    LOG.info("modified " + poolName + "; " + bld.toString());
+    LOG.info("modifyCachePool of " + info.getPoolName() + " successful; "
+        + bld.toString());
   }
 
   /**
@@ -646,28 +756,37 @@ public final class CacheManager {
   public void removeCachePool(String poolName)
       throws IOException {
     assert namesystem.hasWriteLock();
-    CachePoolInfo.validateName(poolName);
-    CachePool pool = cachePools.remove(poolName);
-    if (pool == null) {
-      throw new InvalidRequestException(
-          "Cannot remove non-existent cache pool " + poolName);
-    }
-    // Remove all directives in this pool.
-    Iterator<CacheDirective> iter = pool.getDirectiveList().iterator();
-    while (iter.hasNext()) {
-      CacheDirective directive = iter.next();
-      directivesByPath.remove(directive.getPath());
-      directivesById.remove(directive.getId());
-      iter.remove();
-    }
-    if (monitor != null) {
-      monitor.kick();
+    try {
+      CachePoolInfo.validateName(poolName);
+      CachePool pool = cachePools.remove(poolName);
+      if (pool == null) {
+        throw new InvalidRequestException(
+            "Cannot remove non-existent cache pool " + poolName);
+      }
+      // Remove all directives in this pool.
+      Iterator<CacheDirective> iter = pool.getDirectiveList().iterator();
+      while (iter.hasNext()) {
+        CacheDirective directive = iter.next();
+        directivesByPath.remove(directive.getPath());
+        directivesById.remove(directive.getId());
+        iter.remove();
+      }
+      if (monitor != null) {
+        monitor.setNeedsRescan();
+      }
+    } catch (IOException e) {
+      LOG.info("removeCachePool of " + poolName + " failed: ", e);
+      throw e;
     }
+    LOG.info("removeCachePool of " + poolName + " successful.");
   }
 
   public BatchedListEntries<CachePoolEntry>
       listCachePools(FSPermissionChecker pc, String prevKey) {
     assert namesystem.hasReadLock();
+    if (monitor != null) {
+      monitor.waitForRescanIfNeeded();
+    }
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     ArrayList<CachePoolEntry> results = 
         new ArrayList<CachePoolEntry>(NUM_PRE_ALLOCATED_ENTRIES);
@@ -782,7 +901,7 @@ public final class CacheManager {
    * @param sdPath path of the storage directory
    * @throws IOException
    */
-  public void saveState(DataOutput out, String sdPath)
+  public void saveState(DataOutputStream out, String sdPath)
       throws IOException {
     out.writeLong(nextDirectiveId);
     savePools(out, sdPath);
@@ -805,7 +924,7 @@ public final class CacheManager {
   /**
    * Save cache pools to fsimage
    */
-  private void savePools(DataOutput out,
+  private void savePools(DataOutputStream out,
       String sdPath) throws IOException {
     StartupProgress prog = NameNode.getStartupProgress();
     Step step = new Step(StepType.CACHE_POOLS, sdPath);
@@ -814,7 +933,7 @@ public final class CacheManager {
     Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
     out.writeInt(cachePools.size());
     for (CachePool pool: cachePools.values()) {
-      pool.getInfo(true).writeTo(out);
+      FSImageSerialization.writeCachePoolInfo(out, pool.getInfo(true));
       counter.increment();
     }
     prog.endStep(Phase.SAVING_CHECKPOINT, step);
@@ -823,7 +942,7 @@ public final class CacheManager {
   /*
    * Save cache entries to fsimage
    */
-  private void saveDirectives(DataOutput out, String sdPath)
+  private void saveDirectives(DataOutputStream out, String sdPath)
       throws IOException {
     StartupProgress prog = NameNode.getStartupProgress();
     Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
@@ -832,11 +951,7 @@ public final class CacheManager {
     Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
     out.writeInt(directivesById.size());
     for (CacheDirective directive : directivesById.values()) {
-      out.writeLong(directive.getId());
-      Text.writeString(out, directive.getPath());
-      out.writeShort(directive.getReplication());
-      Text.writeString(out, directive.getPool().getPoolName());
-      out.writeLong(directive.getExpiryTime());
+      FSImageSerialization.writeCacheDirectiveInfo(out, directive.toInfo());
       counter.increment();
     }
     prog.endStep(Phase.SAVING_CHECKPOINT, step);
@@ -854,7 +969,7 @@ public final class CacheManager {
     prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfPools);
     Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
     for (int i = 0; i < numberOfPools; i++) {
-      addCachePool(CachePoolInfo.readFrom(in));
+      addCachePool(FSImageSerialization.readCachePoolInfo(in));
       counter.increment();
     }
     prog.endStep(Phase.LOADING_FSIMAGE, step);
@@ -871,19 +986,17 @@ public final class CacheManager {
     prog.setTotal(Phase.LOADING_FSIMAGE, step, numDirectives);
     Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
     for (int i = 0; i < numDirectives; i++) {
-      long directiveId = in.readLong();
-      String path = Text.readString(in);
-      short replication = in.readShort();
-      String poolName = Text.readString(in);
-      long expiryTime = in.readLong();
+      CacheDirectiveInfo info = FSImageSerialization.readCacheDirectiveInfo(in);
       // Get pool reference by looking it up in the map
+      final String poolName = info.getPool();
       CachePool pool = cachePools.get(poolName);
       if (pool == null) {
         throw new IOException("Directive refers to pool " + poolName +
             ", which does not exist.");
       }
       CacheDirective directive =
-          new CacheDirective(directiveId, path, replication, expiryTime);
+          new CacheDirective(info.getId(), info.getPath().toUri().getPath(),
+              info.getReplication(), info.getExpiration().getAbsoluteMillis());
       boolean addedDirective = pool.getDirectiveList().add(directive);
       assert addedDirective;
       if (directivesById.put(directive.getId(), directive) != null) {

+ 25 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java

@@ -49,8 +49,8 @@ import com.google.common.base.Preconditions;
 public final class CachePool {
   public static final Log LOG = LogFactory.getLog(CachePool.class);
 
-  public static final int DEFAULT_WEIGHT = 100;
-  
+  public static final long DEFAULT_LIMIT = Long.MAX_VALUE;
+
   @Nonnull
   private final String poolName;
 
@@ -71,7 +71,10 @@ public final class CachePool {
   @Nonnull
   private FsPermission mode;
 
-  private int weight;
+  /**
+   * Maximum number of bytes that can be cached in this pool.
+   */
+  private long limit;
 
   private long bytesNeeded;
   private long bytesCached;
@@ -118,10 +121,10 @@ public final class CachePool {
     }
     FsPermission mode = (info.getMode() == null) ? 
         FsPermission.getCachePoolDefault() : info.getMode();
-    Integer weight = (info.getWeight() == null) ?
-        DEFAULT_WEIGHT : info.getWeight();
+    long limit = info.getLimit() == null ?
+        DEFAULT_LIMIT : info.getLimit();
     return new CachePool(info.getPoolName(),
-        ownerName, groupName, mode, weight);
+        ownerName, groupName, mode, limit);
   }
 
   /**
@@ -131,11 +134,11 @@ public final class CachePool {
   static CachePool createFromInfo(CachePoolInfo info) {
     return new CachePool(info.getPoolName(),
         info.getOwnerName(), info.getGroupName(),
-        info.getMode(), info.getWeight());
+        info.getMode(), info.getLimit());
   }
 
   CachePool(String poolName, String ownerName, String groupName,
-      FsPermission mode, int weight) {
+      FsPermission mode, long limit) {
     Preconditions.checkNotNull(poolName);
     Preconditions.checkNotNull(ownerName);
     Preconditions.checkNotNull(groupName);
@@ -144,7 +147,7 @@ public final class CachePool {
     this.ownerName = ownerName;
     this.groupName = groupName;
     this.mode = new FsPermission(mode);
-    this.weight = weight;
+    this.limit = limit;
   }
 
   public String getPoolName() {
@@ -177,16 +180,16 @@ public final class CachePool {
     this.mode = new FsPermission(mode);
     return this;
   }
-  
-  public int getWeight() {
-    return weight;
+
+  public long getLimit() {
+    return limit;
   }
 
-  public CachePool setWeight(int weight) {
-    this.weight = weight;
+  public CachePool setLimit(long bytes) {
+    this.limit = bytes;
     return this;
   }
-  
+
   /**
    * Get either full or partial information about this CachePool.
    *
@@ -204,7 +207,7 @@ public final class CachePool {
     return info.setOwnerName(ownerName).
         setGroupName(groupName).
         setMode(new FsPermission(mode)).
-        setWeight(weight);
+        setLimit(limit);
   }
 
   /**
@@ -241,6 +244,10 @@ public final class CachePool {
     return bytesCached;
   }
 
+  public long getBytesOverlimit() {
+    return Math.max(bytesNeeded-limit, 0);
+  }
+
   public long getFilesNeeded() {
     return filesNeeded;
   }
@@ -258,6 +265,7 @@ public final class CachePool {
     return new CachePoolStats.Builder().
         setBytesNeeded(bytesNeeded).
         setBytesCached(bytesCached).
+        setBytesOverlimit(getBytesOverlimit()).
         setFilesNeeded(filesNeeded).
         setFilesCached(filesCached).
         build();
@@ -291,7 +299,7 @@ public final class CachePool {
         append(", ownerName:").append(ownerName).
         append(", groupName:").append(groupName).
         append(", mode:").append(mode).
-        append(", weight:").append(weight).
+        append(", limit:").append(limit).
         append(" }").toString();
   }
 

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

@@ -160,10 +160,10 @@ public class FSEditLog implements LogsPurgeable {
   private long totalTimeTransactions;  // total time for all transactions
   private NameNodeMetrics metrics;
 
-  private NNStorage storage;
-  private Configuration conf;
+  private final NNStorage storage;
+  private final Configuration conf;
   
-  private List<URI> editsDirs;
+  private final List<URI> editsDirs;
 
   private ThreadLocal<OpInstanceCache> cache =
       new ThreadLocal<OpInstanceCache>() {
@@ -176,7 +176,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * The edit directories that are shared between primary and secondary.
    */
-  private List<URI> sharedEditsDirs;
+  private final List<URI> sharedEditsDirs;
 
   private static class TransactionId {
     public long txid;
@@ -203,10 +203,6 @@ public class FSEditLog implements LogsPurgeable {
    * @param editsDirs List of journals to use
    */
   FSEditLog(Configuration conf, NNStorage storage, List<URI> editsDirs) {
-    init(conf, storage, editsDirs);
-  }
-  
-  private void init(Configuration conf, NNStorage storage, List<URI> editsDirs) {
     isSyncRunning = false;
     this.conf = conf;
     this.storage = storage;

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

@@ -24,12 +24,14 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.Arrays;
 import java.util.EnumMap;
+import java.util.EnumSet;
 import java.util.List;
 
 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.fs.CacheFlag;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -650,7 +652,7 @@ public class FSEditLogLoader {
       ModifyCacheDirectiveInfoOp modifyOp =
           (ModifyCacheDirectiveInfoOp) op;
       fsNamesys.getCacheManager().modifyDirective(
-          modifyOp.directive, null);
+          modifyOp.directive, null, EnumSet.of(CacheFlag.FORCE));
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
       }

+ 33 - 186
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -64,7 +64,6 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Date;
 import java.util.EnumMap;
 import java.util.List;
 import java.util.zip.CheckedInputStream;
@@ -76,7 +75,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -120,11 +118,10 @@ import com.google.common.base.Preconditions;
 @InterfaceStability.Unstable
 public abstract class FSEditLogOp {
   public final FSEditLogOpCodes opCode;
-  long txid;
+  long txid = HdfsConstants.INVALID_TXID;
   byte[] rpcClientId = RpcConstants.DUMMY_CLIENT_ID;
   int rpcCallId = RpcConstants.INVALID_CALL_ID;
 
-  @SuppressWarnings("deprecation")
   final public static class OpInstanceCache {
     private EnumMap<FSEditLogOpCodes, FSEditLogOp> inst = 
         new EnumMap<FSEditLogOpCodes, FSEditLogOp>(FSEditLogOpCodes.class);
@@ -149,13 +146,10 @@ public abstract class FSEditLogOp {
       inst.put(OP_REASSIGN_LEASE, new ReassignLeaseOp());
       inst.put(OP_GET_DELEGATION_TOKEN, new GetDelegationTokenOp());
       inst.put(OP_RENEW_DELEGATION_TOKEN, new RenewDelegationTokenOp());
-      inst.put(OP_CANCEL_DELEGATION_TOKEN, 
-                    new CancelDelegationTokenOp());
+      inst.put(OP_CANCEL_DELEGATION_TOKEN, new CancelDelegationTokenOp());
       inst.put(OP_UPDATE_MASTER_KEY, new UpdateMasterKeyOp());
-      inst.put(OP_START_LOG_SEGMENT,
-                    new LogSegmentOp(OP_START_LOG_SEGMENT));
-      inst.put(OP_END_LOG_SEGMENT,
-                    new LogSegmentOp(OP_END_LOG_SEGMENT));
+      inst.put(OP_START_LOG_SEGMENT, new LogSegmentOp(OP_START_LOG_SEGMENT));
+      inst.put(OP_END_LOG_SEGMENT, new LogSegmentOp(OP_END_LOG_SEGMENT));
       inst.put(OP_UPDATE_BLOCKS, new UpdateBlocksOp());
 
       inst.put(OP_ALLOW_SNAPSHOT, new AllowSnapshotOp());
@@ -165,12 +159,10 @@ public abstract class FSEditLogOp {
       inst.put(OP_RENAME_SNAPSHOT, new RenameSnapshotOp());
       inst.put(OP_SET_GENSTAMP_V2, new SetGenstampV2Op());
       inst.put(OP_ALLOCATE_BLOCK_ID, new AllocateBlockIdOp());
-      inst.put(OP_ADD_CACHE_DIRECTIVE,
-          new AddCacheDirectiveInfoOp());
-      inst.put(OP_MODIFY_CACHE_DIRECTIVE,
-          new ModifyCacheDirectiveInfoOp());
-      inst.put(OP_REMOVE_CACHE_DIRECTIVE,
-          new RemoveCacheDirectiveInfoOp());
+
+      inst.put(OP_ADD_CACHE_DIRECTIVE, new AddCacheDirectiveInfoOp());
+      inst.put(OP_MODIFY_CACHE_DIRECTIVE, new ModifyCacheDirectiveInfoOp());
+      inst.put(OP_REMOVE_CACHE_DIRECTIVE, new RemoveCacheDirectiveInfoOp());
       inst.put(OP_ADD_CACHE_POOL, new AddCachePoolOp());
       inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
       inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
@@ -187,7 +179,6 @@ public abstract class FSEditLogOp {
    */
   private FSEditLogOp(FSEditLogOpCodes opCode) {
     this.opCode = opCode;
-    this.txid = HdfsConstants.INVALID_TXID;
   }
 
   public long getTransactionId() {
@@ -2895,56 +2886,25 @@ public abstract class FSEditLogOp {
 
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
-      long id = FSImageSerialization.readLong(in);
-      String path = FSImageSerialization.readString(in);
-      short replication = FSImageSerialization.readShort(in);
-      String pool = FSImageSerialization.readString(in);
-      long expiryTime = FSImageSerialization.readLong(in);
-      directive = new CacheDirectiveInfo.Builder().
-          setId(id).
-          setPath(new Path(path)).
-          setReplication(replication).
-          setPool(pool).
-          setExpiration(CacheDirectiveInfo.Expiration.newAbsolute(expiryTime)).
-          build();
+      directive = FSImageSerialization.readCacheDirectiveInfo(in);
       readRpcIds(in, logVersion);
     }
 
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
-      FSImageSerialization.writeLong(directive.getId(), out);
-      FSImageSerialization.writeString(directive.getPath().toUri().getPath(), out);
-      FSImageSerialization.writeShort(directive.getReplication(), out);
-      FSImageSerialization.writeString(directive.getPool(), out);
-      FSImageSerialization.writeLong(
-          directive.getExpiration().getMillis(), out);
+      FSImageSerialization.writeCacheDirectiveInfo(out, directive);
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
 
     @Override
     protected void toXml(ContentHandler contentHandler) throws SAXException {
-      XMLUtils.addSaxString(contentHandler, "ID",
-          directive.getId().toString());
-      XMLUtils.addSaxString(contentHandler, "PATH",
-          directive.getPath().toUri().getPath());
-      XMLUtils.addSaxString(contentHandler, "REPLICATION",
-          Short.toString(directive.getReplication()));
-      XMLUtils.addSaxString(contentHandler, "POOL", directive.getPool());
-      XMLUtils.addSaxString(contentHandler, "EXPIRATION",
-          "" + directive.getExpiration().getMillis());
+      FSImageSerialization.writeCacheDirectiveInfo(contentHandler, directive);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
     }
 
     @Override
     void fromXml(Stanza st) throws InvalidXmlException {
-      directive = new CacheDirectiveInfo.Builder().
-          setId(Long.parseLong(st.getValue("ID"))).
-          setPath(new Path(st.getValue("PATH"))).
-          setReplication(Short.parseShort(st.getValue("REPLICATION"))).
-          setPool(st.getValue("POOL")).
-          setExpiration(CacheDirectiveInfo.Expiration.newAbsolute(
-              Long.parseLong(st.getValue("EXPIRATION")))).
-          build();
+      directive = FSImageSerialization.readCacheDirectiveInfo(st);
       readRpcIdsFromXml(st);
     }
 
@@ -2988,104 +2948,25 @@ public abstract class FSEditLogOp {
 
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
-      CacheDirectiveInfo.Builder builder =
-          new CacheDirectiveInfo.Builder();
-      builder.setId(FSImageSerialization.readLong(in));
-      byte flags = in.readByte();
-      if ((flags & 0x1) != 0) {
-        builder.setPath(new Path(FSImageSerialization.readString(in)));
-      }
-      if ((flags & 0x2) != 0) {
-        builder.setReplication(FSImageSerialization.readShort(in));
-      }
-      if ((flags & 0x4) != 0) {
-        builder.setPool(FSImageSerialization.readString(in));
-      }
-      if ((flags & 0x8) != 0) {
-        builder.setExpiration(
-            CacheDirectiveInfo.Expiration.newAbsolute(
-                FSImageSerialization.readLong(in)));
-      }
-      if ((flags & ~0xF) != 0) {
-        throw new IOException("unknown flags set in " +
-            "ModifyCacheDirectiveInfoOp: " + flags);
-      }
-      this.directive = builder.build();
+      this.directive = FSImageSerialization.readCacheDirectiveInfo(in);
       readRpcIds(in, logVersion);
     }
 
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
-      FSImageSerialization.writeLong(directive.getId(), out);
-      byte flags = (byte)(
-          ((directive.getPath() != null) ? 0x1 : 0) |
-          ((directive.getReplication() != null) ? 0x2 : 0) |
-          ((directive.getPool() != null) ? 0x4 : 0) |
-          ((directive.getExpiration() != null) ? 0x8 : 0)
-        );
-      out.writeByte(flags);
-      if (directive.getPath() != null) {
-        FSImageSerialization.writeString(
-            directive.getPath().toUri().getPath(), out);
-      }
-      if (directive.getReplication() != null) {
-        FSImageSerialization.writeShort(directive.getReplication(), out);
-      }
-      if (directive.getPool() != null) {
-        FSImageSerialization.writeString(directive.getPool(), out);
-      }
-      if (directive.getExpiration() != null) {
-        FSImageSerialization.writeLong(directive.getExpiration().getMillis(),
-            out);
-      }
+      FSImageSerialization.writeCacheDirectiveInfo(out, directive);
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
 
     @Override
     protected void toXml(ContentHandler contentHandler) throws SAXException {
-      XMLUtils.addSaxString(contentHandler, "ID",
-          Long.toString(directive.getId()));
-      if (directive.getPath() != null) {
-        XMLUtils.addSaxString(contentHandler, "PATH",
-            directive.getPath().toUri().getPath());
-      }
-      if (directive.getReplication() != null) {
-        XMLUtils.addSaxString(contentHandler, "REPLICATION",
-            Short.toString(directive.getReplication()));
-      }
-      if (directive.getPool() != null) {
-        XMLUtils.addSaxString(contentHandler, "POOL", directive.getPool());
-      }
-      if (directive.getExpiration() != null) {
-        XMLUtils.addSaxString(contentHandler, "EXPIRATION",
-            "" + directive.getExpiration().getMillis());
-      }
+      FSImageSerialization.writeCacheDirectiveInfo(contentHandler, directive);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
     }
 
     @Override
     void fromXml(Stanza st) throws InvalidXmlException {
-      CacheDirectiveInfo.Builder builder =
-          new CacheDirectiveInfo.Builder();
-      builder.setId(Long.parseLong(st.getValue("ID")));
-      String path = st.getValueOrNull("PATH");
-      if (path != null) {
-        builder.setPath(new Path(path));
-      }
-      String replicationString = st.getValueOrNull("REPLICATION");
-      if (replicationString != null) {
-        builder.setReplication(Short.parseShort(replicationString));
-      }
-      String pool = st.getValueOrNull("POOL");
-      if (pool != null) {
-        builder.setPool(pool);
-      }
-      String expiryTime = st.getValueOrNull("EXPIRATION");
-      if (expiryTime != null) {
-        builder.setExpiration(CacheDirectiveInfo.Expiration.newAbsolute(
-            Long.parseLong(expiryTime)));
-      }
-      this.directive = builder.build();
+      this.directive = FSImageSerialization.readCacheDirectiveInfo(st);
       readRpcIdsFromXml(st);
     }
 
@@ -3184,30 +3065,35 @@ public abstract class FSEditLogOp {
 
     public AddCachePoolOp setPool(CachePoolInfo info) {
       this.info = info;
+      assert(info.getPoolName() != null);
+      assert(info.getOwnerName() != null);
+      assert(info.getGroupName() != null);
+      assert(info.getMode() != null);
+      assert(info.getLimit() != null);
       return this;
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
-      info = CachePoolInfo.readFrom(in);
+      info = FSImageSerialization.readCachePoolInfo(in);
       readRpcIds(in, logVersion);
     }
 
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
-      info.writeTo(out);
+      FSImageSerialization.writeCachePoolInfo(out, info);
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
 
     @Override
     protected void toXml(ContentHandler contentHandler) throws SAXException {
-      info.writeXmlTo(contentHandler);
+      FSImageSerialization.writeCachePoolInfo(contentHandler, info);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
     }
 
     @Override
     void fromXml(Stanza st) throws InvalidXmlException {
-      this.info = CachePoolInfo.readXmlFrom(st);
+      this.info = FSImageSerialization.readCachePoolInfo(st);
       readRpcIdsFromXml(st);
     }
 
@@ -3219,7 +3105,7 @@ public abstract class FSEditLogOp {
       builder.append("ownerName=" + info.getOwnerName() + ",");
       builder.append("groupName=" + info.getGroupName() + ",");
       builder.append("mode=" + Short.toString(info.getMode().toShort()) + ",");
-      builder.append("weight=" + Integer.toString(info.getWeight()));
+      builder.append("limit=" + Long.toString(info.getLimit()));
       appendRpcIdsToString(builder, rpcClientId, rpcCallId);
       builder.append("]");
       return builder.toString();
@@ -3245,25 +3131,25 @@ public abstract class FSEditLogOp {
 
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
-      info = CachePoolInfo.readFrom(in);
+      info = FSImageSerialization.readCachePoolInfo(in);
       readRpcIds(in, logVersion);
     }
 
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
-      info.writeTo(out);
+      FSImageSerialization.writeCachePoolInfo(out, info);
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
 
     @Override
     protected void toXml(ContentHandler contentHandler) throws SAXException {
-      cachePoolInfoToXml(contentHandler, info);
+      FSImageSerialization.writeCachePoolInfo(contentHandler, info);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
     }
 
     @Override
     void fromXml(Stanza st) throws InvalidXmlException {
-      this.info = cachePoolInfoFromXml(st);
+      this.info = FSImageSerialization.readCachePoolInfo(st);
       readRpcIdsFromXml(st);
     }
 
@@ -3284,8 +3170,8 @@ public abstract class FSEditLogOp {
       if (info.getMode() != null) {
         fields.add("mode=" + info.getMode().toString());
       }
-      if (info.getWeight() != null) {
-        fields.add("weight=" + info.getWeight());
+      if (info.getLimit() != null) {
+        fields.add("limit=" + info.getLimit());
       }
       builder.append(Joiner.on(",").join(fields));
       appendRpcIdsToString(builder, rpcClientId, rpcCallId);
@@ -3439,9 +3325,7 @@ public abstract class FSEditLogOp {
      * @param in The stream to read from.
      * @param logVersion The version of the data coming from the stream.
      */
-    @SuppressWarnings("deprecation")
-    public Reader(DataInputStream in, StreamLimiter limiter,
-        int logVersion) {
+    public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
       this.logVersion = logVersion;
       if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
         this.checksum = new PureJavaCrc32();
@@ -3757,41 +3641,4 @@ public abstract class FSEditLogOp {
     short mode = Short.valueOf(st.getValue("MODE"));
     return new FsPermission(mode);
   }
-
-  public static void cachePoolInfoToXml(ContentHandler contentHandler,
-      CachePoolInfo info) throws SAXException {
-    XMLUtils.addSaxString(contentHandler, "POOLNAME", info.getPoolName());
-    if (info.getOwnerName() != null) {
-      XMLUtils.addSaxString(contentHandler, "OWNERNAME", info.getOwnerName());
-    }
-    if (info.getGroupName() != null) {
-      XMLUtils.addSaxString(contentHandler, "GROUPNAME", info.getGroupName());
-    }
-    if (info.getMode() != null) {
-      fsPermissionToXml(contentHandler, info.getMode());
-    }
-    if (info.getWeight() != null) {
-      XMLUtils.addSaxString(contentHandler, "WEIGHT",
-          Integer.toString(info.getWeight()));
-    }
-  }
-
-  public static CachePoolInfo cachePoolInfoFromXml(Stanza st)
-      throws InvalidXmlException {
-    String poolName = st.getValue("POOLNAME");
-    CachePoolInfo info = new CachePoolInfo(poolName);
-    if (st.hasChildren("OWNERNAME")) {
-      info.setOwnerName(st.getValue("OWNERNAME"));
-    }
-    if (st.hasChildren("GROUPNAME")) {
-      info.setGroupName(st.getValue("GROUPNAME"));
-    }
-    if (st.hasChildren("MODE")) {
-      info.setMode(FSEditLogOp.fsPermissionFromXml(st));
-    }
-    if (st.hasChildren("WEIGHT")) {
-      info.setWeight(Integer.parseInt(st.getValue("WEIGHT")));
-    }
-    return info;
-  }
 }

+ 13 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java

@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.util.Map;
-import java.util.HashMap;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
@@ -30,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Unstable
 public enum FSEditLogOpCodes {
   // last op code in file
-  OP_INVALID                    ((byte) -1),
   OP_ADD                        ((byte)  0),
   OP_RENAME_OLD                 ((byte)  1), // deprecated operation
   OP_DELETE                     ((byte)  2),
@@ -69,9 +65,12 @@ public enum FSEditLogOpCodes {
   OP_ADD_CACHE_POOL                       ((byte) 35),
   OP_MODIFY_CACHE_POOL                    ((byte) 36),
   OP_REMOVE_CACHE_POOL                    ((byte) 37),
-  OP_MODIFY_CACHE_DIRECTIVE    ((byte) 38);
+  OP_MODIFY_CACHE_DIRECTIVE     ((byte) 38),
+
+  // Note that fromByte(..) depends on OP_INVALID being at the last position.  
+  OP_INVALID                    ((byte) -1);
 
-  private byte opCode;
+  private final byte opCode;
 
   /**
    * Constructor
@@ -91,14 +90,7 @@ public enum FSEditLogOpCodes {
     return opCode;
   }
 
-  private static final Map<Byte, FSEditLogOpCodes> byteToEnum =
-    new HashMap<Byte, FSEditLogOpCodes>();
-
-  static {
-    // initialize byte to enum map
-    for(FSEditLogOpCodes opCode : values())
-      byteToEnum.put(opCode.getOpCode(), opCode);
-  }
+  private static final FSEditLogOpCodes[] VALUES = FSEditLogOpCodes.values();
 
   /**
    * Converts byte to FSEditLogOpCodes enum value
@@ -107,6 +99,12 @@ public enum FSEditLogOpCodes {
    * @return enum with byte value of opCode
    */
   public static FSEditLogOpCodes fromByte(byte opCode) {
-    return byteToEnum.get(opCode);
+    if (opCode == -1) {
+      return OP_INVALID;
+    }
+    if (opCode >= 0 && opCode < OP_INVALID.ordinal()) {
+      return VALUES[opCode];
+    }
+    return null;
   }
 }

+ 205 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -30,6 +30,8 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@@ -38,11 +40,16 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
+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.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
 
 import com.google.common.base.Preconditions;
 
@@ -476,4 +483,202 @@ public class FSImageSerialization {
     }
     return ret;
   }
+
+  public static void writeCacheDirectiveInfo(DataOutputStream out,
+      CacheDirectiveInfo directive) throws IOException {
+    writeLong(directive.getId(), out);
+    int flags =
+        ((directive.getPath() != null) ? 0x1 : 0) |
+        ((directive.getReplication() != null) ? 0x2 : 0) |
+        ((directive.getPool() != null) ? 0x4 : 0) |
+        ((directive.getExpiration() != null) ? 0x8 : 0);
+    out.writeInt(flags);
+    if (directive.getPath() != null) {
+      writeString(directive.getPath().toUri().getPath(), out);
+    }
+    if (directive.getReplication() != null) {
+      writeShort(directive.getReplication(), out);
+    }
+    if (directive.getPool() != null) {
+      writeString(directive.getPool(), out);
+    }
+    if (directive.getExpiration() != null) {
+      writeLong(directive.getExpiration().getMillis(), out);
+    }
+  }
+
+  public static CacheDirectiveInfo readCacheDirectiveInfo(DataInput in)
+      throws IOException {
+    CacheDirectiveInfo.Builder builder =
+        new CacheDirectiveInfo.Builder();
+    builder.setId(readLong(in));
+    int flags = in.readInt();
+    if ((flags & 0x1) != 0) {
+      builder.setPath(new Path(readString(in)));
+    }
+    if ((flags & 0x2) != 0) {
+      builder.setReplication(readShort(in));
+    }
+    if ((flags & 0x4) != 0) {
+      builder.setPool(readString(in));
+    }
+    if ((flags & 0x8) != 0) {
+      builder.setExpiration(
+          CacheDirectiveInfo.Expiration.newAbsolute(readLong(in)));
+    }
+    if ((flags & ~0xF) != 0) {
+      throw new IOException("unknown flags set in " +
+          "ModifyCacheDirectiveInfoOp: " + flags);
+    }
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfo readCacheDirectiveInfo(Stanza st)
+      throws InvalidXmlException {
+    CacheDirectiveInfo.Builder builder =
+        new CacheDirectiveInfo.Builder();
+    builder.setId(Long.parseLong(st.getValue("ID")));
+    String path = st.getValueOrNull("PATH");
+    if (path != null) {
+      builder.setPath(new Path(path));
+    }
+    String replicationString = st.getValueOrNull("REPLICATION");
+    if (replicationString != null) {
+      builder.setReplication(Short.parseShort(replicationString));
+    }
+    String pool = st.getValueOrNull("POOL");
+    if (pool != null) {
+      builder.setPool(pool);
+    }
+    String expiryTime = st.getValueOrNull("EXPIRATION");
+    if (expiryTime != null) {
+      builder.setExpiration(CacheDirectiveInfo.Expiration.newAbsolute(
+          Long.parseLong(expiryTime)));
+    }
+    return builder.build();
+  }
+
+  public static void writeCacheDirectiveInfo(ContentHandler contentHandler,
+      CacheDirectiveInfo directive) throws SAXException {
+    XMLUtils.addSaxString(contentHandler, "ID",
+        Long.toString(directive.getId()));
+    if (directive.getPath() != null) {
+      XMLUtils.addSaxString(contentHandler, "PATH",
+          directive.getPath().toUri().getPath());
+    }
+    if (directive.getReplication() != null) {
+      XMLUtils.addSaxString(contentHandler, "REPLICATION",
+          Short.toString(directive.getReplication()));
+    }
+    if (directive.getPool() != null) {
+      XMLUtils.addSaxString(contentHandler, "POOL", directive.getPool());
+    }
+    if (directive.getExpiration() != null) {
+      XMLUtils.addSaxString(contentHandler, "EXPIRATION",
+          "" + directive.getExpiration().getMillis());
+    }
+  }
+
+  public static void writeCachePoolInfo(DataOutputStream out, CachePoolInfo info)
+      throws IOException {
+    writeString(info.getPoolName(), out);
+
+    final String ownerName = info.getOwnerName();
+    final String groupName = info.getGroupName();
+    final Long limit = info.getLimit();
+    final FsPermission mode = info.getMode();
+
+    boolean hasOwner, hasGroup, hasMode, hasLimit;
+    hasOwner = ownerName != null;
+    hasGroup = groupName != null;
+    hasMode = mode != null;
+    hasLimit = limit != null;
+
+    int flags =
+        (hasOwner ? 0x1 : 0) |
+        (hasGroup ? 0x2 : 0) |
+        (hasMode  ? 0x4 : 0) |
+        (hasLimit ? 0x8 : 0);
+    writeInt(flags, out);
+
+    if (hasOwner) {
+      writeString(ownerName, out);
+    }
+    if (hasGroup) {
+      writeString(groupName, out);
+    }
+    if (hasMode) {
+      mode.write(out);
+    }
+    if (hasLimit) {
+      writeLong(limit, out);
+    }
+  }
+
+  public static CachePoolInfo readCachePoolInfo(DataInput in)
+      throws IOException {
+    String poolName = readString(in);
+    CachePoolInfo info = new CachePoolInfo(poolName);
+    int flags = readInt(in);
+    if ((flags & 0x1) != 0) {
+      info.setOwnerName(readString(in));
+    }
+    if ((flags & 0x2) != 0)  {
+      info.setGroupName(readString(in));
+    }
+    if ((flags & 0x4) != 0) {
+      info.setMode(FsPermission.read(in));
+    }
+    if ((flags & 0x8) != 0) {
+      info.setLimit(readLong(in));
+    }
+    if ((flags & ~0xF) != 0) {
+      throw new IOException("Unknown flag in CachePoolInfo: " + flags);
+    }
+    return info;
+  }
+
+  public static void writeCachePoolInfo(ContentHandler contentHandler,
+      CachePoolInfo info) throws SAXException {
+    XMLUtils.addSaxString(contentHandler, "POOLNAME", info.getPoolName());
+
+    final String ownerName = info.getOwnerName();
+    final String groupName = info.getGroupName();
+    final Long limit = info.getLimit();
+    final FsPermission mode = info.getMode();
+
+    if (ownerName != null) {
+      XMLUtils.addSaxString(contentHandler, "OWNERNAME", ownerName);
+    }
+    if (groupName != null) {
+      XMLUtils.addSaxString(contentHandler, "GROUPNAME", groupName);
+    }
+    if (mode != null) {
+      FSEditLogOp.fsPermissionToXml(contentHandler, mode);
+    }
+    if (limit != null) {
+      XMLUtils.addSaxString(contentHandler, "LIMIT",
+          Long.toString(limit));
+    }
+  }
+
+  public static CachePoolInfo readCachePoolInfo(Stanza st)
+      throws InvalidXmlException {
+    String poolName = st.getValue("POOLNAME");
+    CachePoolInfo info = new CachePoolInfo(poolName);
+    if (st.hasChildren("OWNERNAME")) {
+      info.setOwnerName(st.getValue("OWNERNAME"));
+    }
+    if (st.hasChildren("GROUPNAME")) {
+      info.setGroupName(st.getValue("GROUPNAME"));
+    }
+    if (st.hasChildren("MODE")) {
+      info.setMode(FSEditLogOp.fsPermissionFromXml(st));
+    }
+    if (st.hasChildren("LIMIT")) {
+      info.setLimit(Long.parseLong(st.getValue("LIMIT")));
+    }
+    return info;
+  }
+
 }

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -126,6 +126,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
@@ -7054,8 +7055,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  long addCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  long addCacheDirective(CacheDirectiveInfo directive, EnumSet<CacheFlag> flags)
+      throws IOException {
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = isPermissionEnabled ?
         getPermissionChecker() : null;
@@ -7078,7 +7079,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             "for this operation.");
       }
       CacheDirectiveInfo effectiveDirective = 
-          cacheManager.addDirective(directive, pc);
+          cacheManager.addDirective(directive, pc, flags);
       getEditLog().logAddCacheDirectiveInfo(effectiveDirective,
           cacheEntry != null);
       result = effectiveDirective.getId();
@@ -7096,8 +7097,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return result;
   }
 
-  void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = isPermissionEnabled ?
         getPermissionChecker() : null;
@@ -7113,7 +7114,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new SafeModeException(
             "Cannot add cache directive", safeMode);
       }
-      cacheManager.modifyDirective(directive, pc);
+      cacheManager.modifyDirective(directive, pc, flags);
       getEditLog().logModifyCacheDirectiveInfo(directive,
           cacheEntry != null);
       success = true;

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.util.Time.now;
+
 import java.security.PrivilegedExceptionAction;
 import java.util.*;
 import java.io.*;
@@ -41,6 +43,7 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
@@ -88,6 +91,7 @@ public class GetImageServlet extends HttpServlet {
       final GetImageParams parsedParams = new GetImageParams(request, response);
       final Configuration conf = (Configuration) context
           .getAttribute(JspHelper.CURRENT_CONF);
+      final NameNodeMetrics metrics = NameNode.getNameNodeMetrics();
       
       if (UserGroupInformation.isSecurityEnabled() && 
           !isValidRequestor(context, request.getUserPrincipal().getName(), conf)) {
@@ -128,14 +132,26 @@ public class GetImageServlet extends HttpServlet {
               throw new IOException(errorMessage);
             }
             CheckpointFaultInjector.getInstance().beforeGetImageSetsHeaders();
+            long start = now();
             serveFile(imageFile);
+
+            if (metrics != null) { // Metrics non-null only when used inside name node
+              long elapsed = now() - start;
+              metrics.addGetImage(elapsed);
+            }
           } else if (parsedParams.isGetEdit()) {
             long startTxId = parsedParams.getStartTxId();
             long endTxId = parsedParams.getEndTxId();
             
             File editFile = nnImage.getStorage()
                 .findFinalizedEditsFile(startTxId, endTxId);
+            long start = now();
             serveFile(editFile);
+
+            if (metrics != null) { // Metrics non-null only when used inside name node
+              long elapsed = now() - start;
+              metrics.addGetEdit(elapsed);
+            }
           } else if (parsedParams.isPutImage()) {
             final long txid = parsedParams.getTxId();
 
@@ -159,12 +175,18 @@ public class GetImageServlet extends HttpServlet {
                 UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
               }
               
+              long start = now();
               // issue a HTTP get request to download the new fsimage 
               MD5Hash downloadImageDigest =
                 TransferFsImage.downloadImageToStorage(
                         parsedParams.getInfoServer(conf), txid,
                         nnImage.getStorage(), true);
               nnImage.saveDigestAndRenameCheckpointImage(txid, downloadImageDigest);
+
+              if (metrics != null) { // Metrics non-null only when used inside name node
+                long elapsed = now() - start;
+                metrics.addPutImage(elapsed);
+              }
               
               // Now that we have a new checkpoint, we might be able to
               // remove some old ones.

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

@@ -29,6 +29,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -36,6 +37,7 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -1241,14 +1243,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override
   public long addCacheDirective(
-      CacheDirectiveInfo path) throws IOException {
-    return namesystem.addCacheDirective(path);
+      CacheDirectiveInfo path, EnumSet<CacheFlag> flags) throws IOException {
+    return namesystem.addCacheDirective(path, flags);
   }
 
   @Override
   public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
-    namesystem.modifyCacheDirective(directive);
+      CacheDirectiveInfo directive, EnumSet<CacheFlag> flags) throws IOException {
+    namesystem.modifyCacheDirective(directive, flags);
   }
 
   @Override

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java

@@ -89,6 +89,13 @@ public class NameNodeMetrics {
   @Metric("Time loading FS Image at startup in msec")
   MutableGaugeInt fsImageLoadTime;
 
+  @Metric("GetImageServlet getEdit")
+  MutableRate getEdit;
+  @Metric("GetImageServlet getImage")
+  MutableRate getImage;
+  @Metric("GetImageServlet putImage")
+  MutableRate putImage;
+
   NameNodeMetrics(String processName, String sessionId, int[] intervals) {
     registry.tag(ProcessName, processName).tag(SessionId, sessionId);
     
@@ -251,4 +258,16 @@ public class NameNodeMetrics {
   public void setSafeModeTime(long elapsed) {
     safeModeTime.set((int) elapsed);
   }
+
+  public void addGetEdit(long latency) {
+    getEdit.add(latency);
+  }
+
+  public void addGetImage(long latency) {
+    getImage.add(latency);
+  }
+
+  public void addPutImage(long latency) {
+    putImage.add(latency);
+  }
 }

+ 66 - 49
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.tools;
 
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -25,6 +26,7 @@ import org.apache.commons.lang.WordUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
@@ -135,6 +137,7 @@ public class CacheAdmin extends Configured implements Tool {
     public String getShortUsage() {
       return "[" + getName() +
           " -path <path> -pool <pool-name> " +
+          "[-force] " +
           "[-replication <replication>] [-ttl <time-to-live>]]\n";
     }
 
@@ -146,6 +149,8 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<pool-name>", "The pool to which the directive will be " +
           "added. You must have write permission on the cache pool "
           + "in order to add new directives.");
+      listing.addRow("-force",
+          "Skips checking of cache pool resource limits.");
       listing.addRow("<replication>", "The cache replication factor to use. " +
           "Defaults to 1.");
       listing.addRow("<time-to-live>", "How long the directive is " +
@@ -174,7 +179,7 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
       }
       builder.setPool(poolName);
-
+      boolean force = StringUtils.popOption("-force", args);
       String replicationString =
           StringUtils.popOptionWithArgument("-replication", args);
       if (replicationString != null) {
@@ -201,8 +206,12 @@ public class CacheAdmin extends Configured implements Tool {
         
       DistributedFileSystem dfs = getDFS(conf);
       CacheDirectiveInfo directive = builder.build();
+      EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
+      if (force) {
+        flags.add(CacheFlag.FORCE);
+      }
       try {
-        long id = dfs.addCacheDirective(directive);
+        long id = dfs.addCacheDirective(directive, flags);
         System.out.println("Added cache directive " + id);
       } catch (IOException e) {
         System.err.println(prettifyException(e));
@@ -282,7 +291,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     public String getShortUsage() {
       return "[" + getName() +
-          " -id <id> [-path <path>] [-replication <replication>] " +
+          " -id <id> [-path <path>] [-force] [-replication <replication>] " +
           "[-pool <pool-name>] [-ttl <time-to-live>]]\n";
     }
 
@@ -292,6 +301,8 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<id>", "The ID of the directive to modify (required)");
       listing.addRow("<path>", "A path to cache. The path can be " +
           "a directory or a file. (optional)");
+      listing.addRow("-force",
+          "Skips checking of cache pool resource limits.");
       listing.addRow("<replication>", "The cache replication factor to use. " +
           "(optional)");
       listing.addRow("<pool-name>", "The pool to which the directive will be " +
@@ -322,6 +333,7 @@ public class CacheAdmin extends Configured implements Tool {
         builder.setPath(new Path(path));
         modified = true;
       }
+      boolean force = StringUtils.popOption("-force", args);
       String replicationString =
         StringUtils.popOptionWithArgument("-replication", args);
       if (replicationString != null) {
@@ -357,8 +369,12 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
       }
       DistributedFileSystem dfs = getDFS(conf);
+      EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
+      if (force) {
+        flags.add(CacheFlag.FORCE);
+      }
       try {
-        dfs.modifyCacheDirective(builder.build());
+        dfs.modifyCacheDirective(builder.build(), flags);
         System.out.println("Modified cache directive " + idString);
       } catch (IOException e) {
         System.err.println(prettifyException(e));
@@ -536,7 +552,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     public String getShortUsage() {
       return "[" + NAME + " <name> [-owner <owner>] " +
-          "[-group <group>] [-mode <mode>] [-weight <weight>]]\n";
+          "[-group <group>] [-mode <mode>] [-limit <limit>]]\n";
     }
 
     @Override
@@ -551,11 +567,10 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<mode>", "UNIX-style permissions for the pool. " +
           "Permissions are specified in octal, e.g. 0755. " +
           "By default, this is set to " + String.format("0%03o",
-          FsPermission.getCachePoolDefault().toShort()));
-      listing.addRow("<weight>", "Weight of the pool. " +
-          "This is a relative measure of the importance of the pool used " +
-          "during cache resource management. By default, it is set to " +
-          CachePool.DEFAULT_WEIGHT);
+          FsPermission.getCachePoolDefault().toShort()) + ".");
+      listing.addRow("<limit>", "The maximum number of bytes that can be " +
+          "cached by directives in this pool, in aggregate. By default, " +
+          "no limit is set.");
 
       return getShortUsage() + "\n" +
           "Add a new cache pool.\n\n" + 
@@ -564,33 +579,31 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public int run(Configuration conf, List<String> args) throws IOException {
+      String name = StringUtils.popFirstNonOption(args);
+      if (name == null) {
+        System.err.println("You must specify a name when creating a " +
+            "cache pool.");
+        return 1;
+      }
+      CachePoolInfo info = new CachePoolInfo(name);
+
       String owner = StringUtils.popOptionWithArgument("-owner", args);
-      if (owner == null) {
-        owner = UserGroupInformation.getCurrentUser().getShortUserName();
+      if (owner != null) {
+        info.setOwnerName(owner);
       }
       String group = StringUtils.popOptionWithArgument("-group", args);
-      if (group == null) {
-        group = UserGroupInformation.getCurrentUser().getGroupNames()[0];
+      if (group != null) {
+        info.setGroupName(group);
       }
       String modeString = StringUtils.popOptionWithArgument("-mode", args);
-      int mode;
-      if (modeString == null) {
-        mode = FsPermission.getCachePoolDefault().toShort();
-      } else {
-        mode = Integer.parseInt(modeString, 8);
-      }
-      String weightString = StringUtils.popOptionWithArgument("-weight", args);
-      int weight;
-      if (weightString == null) {
-        weight = CachePool.DEFAULT_WEIGHT;
-      } else {
-        weight = Integer.parseInt(weightString);
+      if (modeString != null) {
+        short mode = Short.parseShort(modeString, 8);
+        info.setMode(new FsPermission(mode));
       }
-      String name = StringUtils.popFirstNonOption(args);
-      if (name == null) {
-        System.err.println("You must specify a name when creating a " +
-            "cache pool.");
-        return 1;
+      String limitString = StringUtils.popOptionWithArgument("-limit", args);
+      if (limitString != null) {
+        long limit = Long.parseLong(limitString);
+        info.setLimit(limit);
       }
       if (!args.isEmpty()) {
         System.err.print("Can't understand arguments: " +
@@ -599,11 +612,6 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
       }
       DistributedFileSystem dfs = getDFS(conf);
-      CachePoolInfo info = new CachePoolInfo(name).
-          setOwnerName(owner).
-          setGroupName(group).
-          setMode(new FsPermission((short)mode)).
-          setWeight(weight);
       try {
         dfs.addCachePool(info);
       } catch (IOException e) {
@@ -624,7 +632,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     public String getShortUsage() {
       return "[" + getName() + " <name> [-owner <owner>] " +
-          "[-group <group>] [-mode <mode>] [-weight <weight>]]\n";
+          "[-group <group>] [-mode <mode>] [-limit <limit>]]\n";
     }
 
     @Override
@@ -635,11 +643,12 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<owner>", "Username of the owner of the pool");
       listing.addRow("<group>", "Groupname of the group of the pool.");
       listing.addRow("<mode>", "Unix-style permissions of the pool in octal.");
-      listing.addRow("<weight>", "Weight of the pool.");
+      listing.addRow("<limit>", "Maximum number of bytes that can be cached " +
+          "by this pool.");
 
       return getShortUsage() + "\n" +
           WordUtils.wrap("Modifies the metadata of an existing cache pool. " +
-          "See usage of " + AddCachePoolCommand.NAME + " for more details",
+          "See usage of " + AddCachePoolCommand.NAME + " for more details.",
           MAX_LINE_WIDTH) + "\n\n" +
           listing.toString();
     }
@@ -651,9 +660,9 @@ public class CacheAdmin extends Configured implements Tool {
       String modeString = StringUtils.popOptionWithArgument("-mode", args);
       Integer mode = (modeString == null) ?
           null : Integer.parseInt(modeString, 8);
-      String weightString = StringUtils.popOptionWithArgument("-weight", args);
-      Integer weight = (weightString == null) ?
-          null : Integer.parseInt(weightString);
+      String limitString = StringUtils.popOptionWithArgument("-limit", args);
+      Long limit = (limitString == null) ?
+          null : Long.parseLong(limitString);
       String name = StringUtils.popFirstNonOption(args);
       if (name == null) {
         System.err.println("You must specify a name when creating a " +
@@ -680,8 +689,8 @@ public class CacheAdmin extends Configured implements Tool {
         info.setMode(new FsPermission(mode.shortValue()));
         changed = true;
       }
-      if (weight != null) {
-        info.setWeight(weight);
+      if (limit != null) {
+        info.setLimit(limit);
         changed = true;
       }
       if (!changed) {
@@ -709,8 +718,8 @@ public class CacheAdmin extends Configured implements Tool {
         System.out.print(prefix + "mode " + new FsPermission(mode.shortValue()));
         prefix = " and ";
       }
-      if (weight != null) {
-        System.out.print(prefix + "weight " + weight);
+      if (limit != null) {
+        System.out.print(prefix + "limit " + limit);
         prefix = " and ";
       }
       System.out.print("\n");
@@ -804,11 +813,12 @@ public class CacheAdmin extends Configured implements Tool {
           addField("OWNER", Justification.LEFT).
           addField("GROUP", Justification.LEFT).
           addField("MODE", Justification.LEFT).
-          addField("WEIGHT", Justification.RIGHT);
+          addField("LIMIT", Justification.RIGHT);
       if (printStats) {
         builder.
             addField("BYTES_NEEDED", Justification.RIGHT).
             addField("BYTES_CACHED", Justification.RIGHT).
+            addField("BYTES_OVERLIMIT", Justification.RIGHT).
             addField("FILES_NEEDED", Justification.RIGHT).
             addField("FILES_CACHED", Justification.RIGHT);
       }
@@ -825,12 +835,19 @@ public class CacheAdmin extends Configured implements Tool {
             row.add(info.getOwnerName());
             row.add(info.getGroupName());
             row.add(info.getMode() != null ? info.getMode().toString() : null);
-            row.add(
-                info.getWeight() != null ? info.getWeight().toString() : null);
+            Long limit = info.getLimit();
+            String limitString;
+            if (limit != null && limit.equals(CachePool.DEFAULT_LIMIT)) {
+              limitString = "unlimited";
+            } else {
+              limitString = "" + limit;
+            }
+            row.add(limitString);
             if (printStats) {
               CachePoolStats stats = entry.getStats();
               row.add(Long.toString(stats.getBytesNeeded()));
               row.add(Long.toString(stats.getBytesCached()));
+              row.add(Long.toString(stats.getBytesOverlimit()));
               row.add(Long.toString(stats.getFilesNeeded()));
               row.add(Long.toString(stats.getFilesCached()));
             }

+ 10 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -386,8 +386,13 @@ message CacheDirectiveStatsProto {
   required bool hasExpired = 5;
 }
 
+enum CacheFlagProto {
+  FORCE = 0x01;    // Ignore pool resource limits
+}
+
 message AddCacheDirectiveRequestProto {
   required CacheDirectiveInfoProto info = 1;
+  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
 }
 
 message AddCacheDirectiveResponseProto {
@@ -396,6 +401,7 @@ message AddCacheDirectiveResponseProto {
 
 message ModifyCacheDirectiveRequestProto {
   required CacheDirectiveInfoProto info = 1;
+  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
 }
 
 message ModifyCacheDirectiveResponseProto {
@@ -428,14 +434,15 @@ message CachePoolInfoProto {
   optional string ownerName = 2;
   optional string groupName = 3;
   optional int32 mode = 4;
-  optional int32 weight = 5;
+  optional int64 limit = 5;
 }
 
 message CachePoolStatsProto {
   required int64 bytesNeeded = 1;
   required int64 bytesCached = 2;
-  required int64 filesNeeded = 3;
-  required int64 filesCached = 4;
+  required int64 bytesOverlimit = 3;
+  required int64 filesNeeded = 4;
+  required int64 filesCached = 5;
 }
 
 message AddCachePoolRequestProto {

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -1394,12 +1394,15 @@
   <name>dfs.client.cache.readahead</name>
   <value></value>
   <description>
-    Just like dfs.datanode.readahead.bytes, this setting causes the datanode to
+    When using remote reads, this setting causes the datanode to
     read ahead in the block file using posix_fadvise, potentially decreasing
     I/O wait times.  Unlike dfs.datanode.readahead.bytes, this is a client-side
     setting rather than a setting for the entire datanode.  If present, this
     setting will override the DataNode default.
 
+    When using local reads, this setting determines how much readahead we do in
+    BlockReaderLocal.
+
     If the native libraries are not available to the DataNode, this
     configuration has no effect.
   </description>

+ 18 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.util.VersionInfo;
 
 import java.io.*;
 import java.net.*;
+import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.*;
 import java.util.concurrent.TimeoutException;
@@ -346,9 +347,15 @@ public class DFSTestUtil {
         // Swallow exceptions
       }
       System.out.println("Waiting for "+corruptRepls+" corrupt replicas");
-      repls = ns.getBlockManager().numCorruptReplicas(b.getLocalBlock());
       count++;
-      Thread.sleep(1000);
+      // check more often so corrupt block reports are not easily missed
+      for (int i = 0; i < 10; i++) {
+        repls = ns.getBlockManager().numCorruptReplicas(b.getLocalBlock());
+        Thread.sleep(100);
+        if (repls == corruptRepls) {
+          break;
+        }
+      }
     }
     if (count == ATTEMPTS) {
       throw new TimeoutException("Timed out waiting for corrupt replicas."
@@ -1036,20 +1043,20 @@ public class DFSTestUtil {
     // OP_ADD_CACHE_POOL
     filesystem.addCachePool(new CachePoolInfo("pool1"));
     // OP_MODIFY_CACHE_POOL
-    filesystem.modifyCachePool(new CachePoolInfo("pool1").setWeight(99));
+    filesystem.modifyCachePool(new CachePoolInfo("pool1").setLimit(99l));
     // OP_ADD_PATH_BASED_CACHE_DIRECTIVE
     long id = filesystem.addCacheDirective(
         new CacheDirectiveInfo.Builder().
             setPath(new Path("/path")).
             setReplication((short)1).
             setPool("pool1").
-            build());
+            build(), EnumSet.of(CacheFlag.FORCE));
     // OP_MODIFY_PATH_BASED_CACHE_DIRECTIVE
     filesystem.modifyCacheDirective(
         new CacheDirectiveInfo.Builder().
             setId(id).
             setReplication((short)2).
-            build());
+            build(), EnumSet.of(CacheFlag.FORCE));
     // OP_REMOVE_PATH_BASED_CACHE_DIRECTIVE
     filesystem.removeCacheDirective(id);
     // OP_REMOVE_CACHE_POOL
@@ -1059,4 +1066,10 @@ public class DFSTestUtil {
   public static void abortStream(DFSOutputStream out) throws IOException {
     out.abort();
   }
+
+  public static byte[] asArray(ByteBuffer buf) {
+    byte arr[] = new byte[buf.remaining()];
+    buf.duplicate().get(arr);
+    return arr;
+  }
 }

+ 310 - 46
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java

@@ -32,6 +32,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+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.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -92,22 +94,35 @@ public class TestBlockReaderLocal {
     }
   }
 
-  private static interface BlockReaderLocalTest {
-    final int TEST_LENGTH = 12345;
+  private static class BlockReaderLocalTest {
+    final static int TEST_LENGTH = 12345;
+    final static int BYTES_PER_CHECKSUM = 512;
+
+    public void setConfiguration(HdfsConfiguration conf) {
+      // default: no-op
+    }
     public void setup(File blockFile, boolean usingChecksums)
-        throws IOException;
+        throws IOException {
+      // default: no-op
+    }
     public void doTest(BlockReaderLocal reader, byte original[])
-        throws IOException;
+        throws IOException {
+      // default: no-op
+    }
   }
   
   public void runBlockReaderLocalTest(BlockReaderLocalTest test,
-      boolean checksum) throws IOException {
+      boolean checksum, long readahead) throws IOException {
     MiniDFSCluster cluster = null;
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.
         DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, !checksum);
+    conf.setLong(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
+        BlockReaderLocalTest.BYTES_PER_CHECKSUM);
     conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
-    FileInputStream dataIn = null, checkIn = null;
+    conf.setLong(DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD, readahead);
+    test.setConfiguration(conf);
+    FileInputStream dataIn = null, metaIn = null;
     final Path TEST_PATH = new Path("/a");
     final long RANDOM_SEED = 4567L;
     BlockReaderLocal blockReaderLocal = null;
@@ -143,45 +158,51 @@ public class TestBlockReaderLocal {
       cluster.shutdown();
       cluster = null;
       test.setup(dataFile, checksum);
-      dataIn = new FileInputStream(dataFile);
-      checkIn = new FileInputStream(metaFile);
-      blockReaderLocal = new BlockReaderLocal(new DFSClient.Conf(conf),
-          TEST_PATH.getName(), block, 0, -1,
-          dataIn, checkIn, datanodeID, checksum, null);
+      FileInputStream streams[] = {
+          new FileInputStream(dataFile),
+          new FileInputStream(metaFile)
+      };
+      dataIn = streams[0];
+      metaIn = streams[1];
+      blockReaderLocal = new BlockReaderLocal.Builder(
+              new DFSClient.Conf(conf)).
+          setFilename(TEST_PATH.getName()).
+          setBlock(block).
+          setStreams(streams).
+          setDatanodeID(datanodeID).
+          setCachingStrategy(new CachingStrategy(false, readahead)).
+          setVerifyChecksum(checksum).
+          setBlockMetadataHeader(BlockMetadataHeader.preadHeader(
+              metaIn.getChannel())).
+          build();
       dataIn = null;
-      checkIn = null;
+      metaIn = null;
       test.doTest(blockReaderLocal, original);
+      // BlockReaderLocal should not alter the file position.
+      Assert.assertEquals(0, streams[0].getChannel().position());
+      Assert.assertEquals(0, streams[1].getChannel().position());
     } finally {
       if (fsIn != null) fsIn.close();
       if (fs != null) fs.close();
       if (cluster != null) cluster.shutdown();
       if (dataIn != null) dataIn.close();
-      if (checkIn != null) checkIn.close();
+      if (metaIn != null) metaIn.close();
       if (blockReaderLocal != null) blockReaderLocal.close();
     }
   }
   
   private static class TestBlockReaderLocalImmediateClose 
-      implements BlockReaderLocalTest {
-    @Override
-    public void setup(File blockFile, boolean usingChecksums)
-        throws IOException { }
-    @Override
-    public void doTest(BlockReaderLocal reader, byte original[]) 
-        throws IOException { }
+      extends BlockReaderLocalTest {
   }
   
   @Test
   public void testBlockReaderLocalImmediateClose() throws IOException {
-    runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), true);
-    runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), false);
+    runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), true, 0);
+    runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), false, 0);
   }
   
   private static class TestBlockReaderSimpleReads 
-      implements BlockReaderLocalTest {
-    @Override
-    public void setup(File blockFile, boolean usingChecksums)
-        throws IOException { }
+      extends BlockReaderLocalTest {
     @Override
     public void doTest(BlockReaderLocal reader, byte original[]) 
         throws IOException {
@@ -194,24 +215,43 @@ public class TestBlockReaderLocal {
       assertArrayRegionsEqual(original, 1024, buf, 1024, 513);
       reader.readFully(buf, 1537, 514);
       assertArrayRegionsEqual(original, 1537, buf, 1537, 514);
+      // Readahead is always at least the size of one chunk in this test.
+      Assert.assertTrue(reader.getMaxReadaheadLength() >=
+          BlockReaderLocalTest.BYTES_PER_CHECKSUM);
     }
   }
   
   @Test
   public void testBlockReaderSimpleReads() throws IOException {
-    runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true);
+    runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true,
+        DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderSimpleReadsShortReadahead() throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true,
+        BlockReaderLocalTest.BYTES_PER_CHECKSUM - 1);
   }
 
   @Test
   public void testBlockReaderSimpleReadsNoChecksum() throws IOException {
-    runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false);
+    runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false,
+        DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderSimpleReadsNoReadahead() throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true, 0);
+  }
+
+  @Test
+  public void testBlockReaderSimpleReadsNoChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false, 0);
   }
   
   private static class TestBlockReaderLocalArrayReads2 
-      implements BlockReaderLocalTest {
-    @Override
-    public void setup(File blockFile, boolean usingChecksums)
-        throws IOException { }
+      extends BlockReaderLocalTest {
     @Override
     public void doTest(BlockReaderLocal reader, byte original[]) 
         throws IOException {
@@ -234,21 +274,30 @@ public class TestBlockReaderLocal {
   @Test
   public void testBlockReaderLocalArrayReads2() throws IOException {
     runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
-        true);
+        true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
   }
 
   @Test
   public void testBlockReaderLocalArrayReads2NoChecksum()
       throws IOException {
     runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
-        false);
+        false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalArrayReads2NoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), true, 0);
+  }
+
+  @Test
+  public void testBlockReaderLocalArrayReads2NoChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), false, 0);
   }
 
   private static class TestBlockReaderLocalByteBufferReads 
-      implements BlockReaderLocalTest {
-    @Override
-    public void setup(File blockFile, boolean usingChecksums)
-        throws IOException { }
+      extends BlockReaderLocalTest {
     @Override
     public void doTest(BlockReaderLocal reader, byte original[]) 
         throws IOException {
@@ -268,19 +317,105 @@ public class TestBlockReaderLocal {
   @Test
   public void testBlockReaderLocalByteBufferReads()
       throws IOException {
-    runBlockReaderLocalTest(
-        new TestBlockReaderLocalByteBufferReads(), true);
+    runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
+        true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
   }
 
   @Test
   public void testBlockReaderLocalByteBufferReadsNoChecksum()
       throws IOException {
     runBlockReaderLocalTest(
-        new TestBlockReaderLocalByteBufferReads(), false);
+        new TestBlockReaderLocalByteBufferReads(),
+        false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+  
+  @Test
+  public void testBlockReaderLocalByteBufferReadsNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
+        true, 0);
+  }
+
+  @Test
+  public void testBlockReaderLocalByteBufferReadsNoChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
+        false, 0);
+  }
+
+  /**
+   * Test reads that bypass the bounce buffer (because they are aligned
+   * and bigger than the readahead).
+   */
+  private static class TestBlockReaderLocalByteBufferFastLaneReads 
+      extends BlockReaderLocalTest {
+    @Override
+    public void doTest(BlockReaderLocal reader, byte original[])
+        throws IOException {
+      ByteBuffer buf = ByteBuffer.allocateDirect(TEST_LENGTH);
+      readFully(reader, buf, 0, 5120);
+      buf.flip();
+      assertArrayRegionsEqual(original, 0,
+          DFSTestUtil.asArray(buf), 0,
+          5120);
+      reader.skip(1537);
+      readFully(reader, buf, 0, 1);
+      buf.flip();
+      assertArrayRegionsEqual(original, 6657,
+          DFSTestUtil.asArray(buf), 0,
+          1);
+      reader.setMlocked(true);
+      readFully(reader, buf, 0, 5120);
+      buf.flip();
+      assertArrayRegionsEqual(original, 6658,
+          DFSTestUtil.asArray(buf), 0,
+          5120);
+      reader.setMlocked(false);
+      readFully(reader, buf, 0, 513);
+      buf.flip();
+      assertArrayRegionsEqual(original, 11778,
+          DFSTestUtil.asArray(buf), 0,
+          513);
+      reader.skip(3);
+      readFully(reader, buf, 0, 50);
+      buf.flip();
+      assertArrayRegionsEqual(original, 12294,
+          DFSTestUtil.asArray(buf), 0,
+          50);
+    }
   }
   
+  @Test
+  public void testBlockReaderLocalByteBufferFastLaneReads()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferFastLaneReads(),
+        true, 2 * BlockReaderLocalTest.BYTES_PER_CHECKSUM);
+  }
+
+  @Test
+  public void testBlockReaderLocalByteBufferFastLaneReadsNoChecksum()
+      throws IOException {
+    runBlockReaderLocalTest(
+        new TestBlockReaderLocalByteBufferFastLaneReads(),
+        false, 2 * BlockReaderLocalTest.BYTES_PER_CHECKSUM);
+  }
+
+  @Test
+  public void testBlockReaderLocalByteBufferFastLaneReadsNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferFastLaneReads(),
+        true, 0);
+  }
+
+  @Test
+  public void testBlockReaderLocalByteBufferFastLaneReadsNoChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferFastLaneReads(),
+        false, 0);
+  }
+
   private static class TestBlockReaderLocalReadCorruptStart
-      implements BlockReaderLocalTest {
+      extends BlockReaderLocalTest {
     boolean usingChecksums = false;
     @Override
     public void setup(File blockFile, boolean usingChecksums)
@@ -314,11 +449,12 @@ public class TestBlockReaderLocal {
   @Test
   public void testBlockReaderLocalReadCorruptStart()
       throws IOException {
-    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true);
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true,
+        DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
   }
   
   private static class TestBlockReaderLocalReadCorrupt
-      implements BlockReaderLocalTest {
+      extends BlockReaderLocalTest {
     boolean usingChecksums = false;
     @Override
     public void setup(File blockFile, boolean usingChecksums) 
@@ -364,8 +500,136 @@ public class TestBlockReaderLocal {
   @Test
   public void testBlockReaderLocalReadCorrupt()
       throws IOException {
-    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true);
-    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false);
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true,
+        DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalReadCorruptNoChecksum()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false,
+        DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalReadCorruptNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true, 0);
+  }
+
+  @Test
+  public void testBlockReaderLocalReadCorruptNoChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false, 0);
+  }
+
+  private static class TestBlockReaderLocalWithMlockChanges
+      extends BlockReaderLocalTest {
+    @Override
+    public void setup(File blockFile, boolean usingChecksums)
+        throws IOException {
+    }
+    
+    @Override
+    public void doTest(BlockReaderLocal reader, byte original[])
+        throws IOException {
+      ByteBuffer buf = ByteBuffer.wrap(new byte[TEST_LENGTH]);
+      reader.skip(1);
+      readFully(reader, buf, 1, 9);
+      assertArrayRegionsEqual(original, 1, buf.array(), 1, 9);
+      readFully(reader, buf, 10, 100);
+      assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
+      reader.setMlocked(true);
+      readFully(reader, buf, 110, 700);
+      assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
+      reader.setMlocked(false);
+      reader.skip(1); // skip from offset 810 to offset 811
+      readFully(reader, buf, 811, 5);
+      assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
+    }
+  }
+
+  @Test
+  public void testBlockReaderLocalWithMlockChanges()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
+        true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalWithMlockChangesNoChecksum()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
+        false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalWithMlockChangesNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
+        true, 0);
+  }
+
+  @Test
+  public void testBlockReaderLocalWithMlockChangesNoChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
+        false, 0);
+  }
+
+  private static class TestBlockReaderLocalOnFileWithoutChecksum
+      extends BlockReaderLocalTest {
+    @Override
+    public void setConfiguration(HdfsConfiguration conf) {
+      conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "NULL");
+    }
+
+    @Override
+    public void doTest(BlockReaderLocal reader, byte original[])
+        throws IOException {
+      Assert.assertTrue(!reader.getVerifyChecksum());
+      ByteBuffer buf = ByteBuffer.wrap(new byte[TEST_LENGTH]);
+      reader.skip(1);
+      readFully(reader, buf, 1, 9);
+      assertArrayRegionsEqual(original, 1, buf.array(), 1, 9);
+      readFully(reader, buf, 10, 100);
+      assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
+      reader.setMlocked(true);
+      readFully(reader, buf, 110, 700);
+      assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
+      reader.setMlocked(false);
+      reader.skip(1); // skip from offset 810 to offset 811
+      readFully(reader, buf, 811, 5);
+      assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
+    }
+  }
+
+  @Test
+  public void testBlockReaderLocalOnFileWithoutChecksum()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
+        true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalOnFileWithoutChecksumNoChecksum()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
+        false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalOnFileWithoutChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
+        true, 0);
+  }
+
+  @Test
+  public void testBlockReaderLocalOnFileWithoutChecksumNoChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
+        false, 0);
   }
 
   @Test(timeout=60000)

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

@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.security.token.Token;
 import org.junit.Assert;
@@ -138,7 +139,8 @@ public class TestConnCache {
                            Matchers.anyLong(),
                            Matchers.anyInt(),
                            Matchers.anyBoolean(),
-                           Matchers.anyString());
+                           Matchers.anyString(),
+                           (CachingStrategy)Matchers.anyObject());
 
     // Initial read
     pread(in, 0, dataBuf, 0, dataBuf.length, authenticData);

+ 81 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -336,6 +336,58 @@ public class TestDecommission {
     testDecommission(1, 6);
   }
   
+  /**
+   * Tests decommission with replicas on the target datanode cannot be migrated
+   * to other datanodes and satisfy the replication factor. Make sure the
+   * datanode won't get stuck in decommissioning state.
+   */
+  @Test(timeout = 360000)
+  public void testDecommission2() throws IOException {
+    LOG.info("Starting test testDecommission");
+    int numNamenodes = 1;
+    int numDatanodes = 4;
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
+    startCluster(numNamenodes, numDatanodes, conf);
+
+    ArrayList<ArrayList<DatanodeInfo>> namenodeDecomList = new ArrayList<ArrayList<DatanodeInfo>>(
+        numNamenodes);
+    namenodeDecomList.add(0, new ArrayList<DatanodeInfo>(numDatanodes));
+
+    Path file1 = new Path("testDecommission2.dat");
+    int replicas = 4;
+
+    // Start decommissioning one namenode at a time
+    ArrayList<DatanodeInfo> decommissionedNodes = namenodeDecomList.get(0);
+    FileSystem fileSys = cluster.getFileSystem(0);
+    FSNamesystem ns = cluster.getNamesystem(0);
+
+    writeFile(fileSys, file1, replicas);
+
+    int deadDecomissioned = ns.getNumDecomDeadDataNodes();
+    int liveDecomissioned = ns.getNumDecomLiveDataNodes();
+
+    // Decommission one node. Verify that node is decommissioned.
+    DatanodeInfo decomNode = decommissionNode(0, decommissionedNodes,
+        AdminStates.DECOMMISSIONED);
+    decommissionedNodes.add(decomNode);
+    assertEquals(deadDecomissioned, ns.getNumDecomDeadDataNodes());
+    assertEquals(liveDecomissioned + 1, ns.getNumDecomLiveDataNodes());
+
+    // Ensure decommissioned datanode is not automatically shutdown
+    DFSClient client = getDfsClient(cluster.getNameNode(0), conf);
+    assertEquals("All datanodes must be alive", numDatanodes,
+        client.datanodeReport(DatanodeReportType.LIVE).length);
+    assertNull(checkFile(fileSys, file1, replicas, decomNode.getXferAddr(),
+        numDatanodes));
+    cleanupFile(fileSys, file1);
+
+    // Restart the cluster and ensure recommissioned datanodes
+    // are allowed to register with the namenode
+    cluster.shutdown();
+    startCluster(1, 4, conf);
+    cluster.shutdown();
+  }
+  
   /**
    * Tests recommission for non federated cluster
    */
@@ -388,7 +440,20 @@ public class TestDecommission {
         DFSClient client = getDfsClient(cluster.getNameNode(i), conf);
         assertEquals("All datanodes must be alive", numDatanodes, 
             client.datanodeReport(DatanodeReportType.LIVE).length);
-        assertNull(checkFile(fileSys, file1, replicas, decomNode.getXferAddr(), numDatanodes));
+        // wait for the block to be replicated
+        int tries = 0;
+        while (tries++ < 20) {
+          try {
+            Thread.sleep(1000);
+            if (checkFile(fileSys, file1, replicas, decomNode.getXferAddr(),
+                numDatanodes) == null) {
+              break;
+            }
+          } catch (InterruptedException ie) {
+          }
+        }
+        assertTrue("Checked if block was replicated after decommission, tried "
+            + tries + " times.", tries < 20);
         cleanupFile(fileSys, file1);
       }
     }
@@ -429,12 +494,25 @@ public class TestDecommission {
       DFSClient client = getDfsClient(cluster.getNameNode(i), conf);
       assertEquals("All datanodes must be alive", numDatanodes, 
           client.datanodeReport(DatanodeReportType.LIVE).length);
-      assertNull(checkFile(fileSys, file1, replicas, decomNode.getXferAddr(), numDatanodes));
+      int tries =0;
+      // wait for the block to be replicated
+      while (tries++ < 20) {
+        try {
+          Thread.sleep(1000);
+          if (checkFile(fileSys, file1, replicas, decomNode.getXferAddr(),
+              numDatanodes) == null) {
+            break;
+          }
+        } catch (InterruptedException ie) {
+        }
+      }
+      assertTrue("Checked if block was replicated after decommission, tried "
+          + tries + " times.", tries < 20);
 
       // stop decommission and check if the new replicas are removed
       recomissionNode(decomNode);
       // wait for the block to be deleted
-      int tries = 0;
+      tries = 0;
       while (tries++ < 20) {
         try {
           Thread.sleep(1000);

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

@@ -259,7 +259,6 @@ public class TestShortCircuitLocalRead {
       assertTrue("/ should be a directory", fs.getFileStatus(path)
           .isDirectory() == true);
       
-      // create a new file in home directory. Do not close it.
       byte[] fileData = AppendTestUtil.randomBytes(seed, size);
       Path file1 = fs.makeQualified(new Path("filelocal.dat"));
       FSDataOutputStream stm = createFile(fs, file1, 1);

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

@@ -239,7 +239,7 @@ public class OfflineEditsViewerHelper {
         .setOwnerName("carlton")
         .setGroupName("party")
         .setMode(new FsPermission((short)0700))
-        .setWeight(1989));
+        .setLimit(1989l));
     // OP_ADD_PATH_BASED_CACHE_DIRECTIVE 33
     long id = dfs.addCacheDirective(
         new CacheDirectiveInfo.Builder().

+ 342 - 305
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java

@@ -34,6 +34,7 @@ import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Date;
+import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
@@ -43,6 +44,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
@@ -92,25 +94,49 @@ public class TestCacheDirectives {
   static private MiniDFSCluster cluster;
   static private DistributedFileSystem dfs;
   static private NamenodeProtocols proto;
+  static private NameNode namenode;
   static private CacheManipulator prevCacheManipulator;
 
   static {
+    NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
     EditLogFileOutputStream.setShouldSkipFsyncForTesting(false);
   }
 
-  @Before
-  public void setup() throws Exception {
-    conf = new HdfsConfiguration();
+  private static final long BLOCK_SIZE = 512;
+  private static final int NUM_DATANODES = 4;
+  // Most Linux installs will allow non-root users to lock 64KB.
+  // In this test though, we stub out mlock so this doesn't matter.
+  private static final long CACHE_CAPACITY = 64 * 1024 / NUM_DATANODES;
+
+  private static HdfsConfiguration createCachingConf() {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, CACHE_CAPACITY);
+    conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, true);
+    conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000);
+    conf.setLong(DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1000);
     // set low limits here for testing purposes
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES, 2);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES, 2);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES,
+        2);
+
+    return conf;
+  }
+
+  @Before
+  public void setup() throws Exception {
+    conf = createCachingConf();
+    cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
     proto = cluster.getNameNodeRpc();
+    namenode = cluster.getNameNode();
     prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
     NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
-    LogManager.getLogger(CacheReplicationMonitor.class).setLevel(Level.TRACE);
+    LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel(
+        Level.TRACE);
   }
 
   @After
@@ -127,7 +153,7 @@ public class TestCacheDirectives {
     final String poolName = "pool1";
     CachePoolInfo info = new CachePoolInfo(poolName).
         setOwnerName("bob").setGroupName("bobgroup").
-        setMode(new FsPermission((short)0755)).setWeight(150);
+        setMode(new FsPermission((short)0755)).setLimit(150l);
 
     // Add a pool
     dfs.addCachePool(info);
@@ -168,7 +194,7 @@ public class TestCacheDirectives {
 
     // Modify the pool
     info.setOwnerName("jane").setGroupName("janegroup")
-        .setMode(new FsPermission((short)0700)).setWeight(314);
+        .setMode(new FsPermission((short)0700)).setLimit(314l);
     dfs.modifyCachePool(info);
 
     // Do some invalid modify pools
@@ -263,10 +289,10 @@ public class TestCacheDirectives {
     String ownerName = "abc";
     String groupName = "123";
     FsPermission mode = new FsPermission((short)0755);
-    int weight = 150;
+    long limit = 150;
     dfs.addCachePool(new CachePoolInfo(poolName).
         setOwnerName(ownerName).setGroupName(groupName).
-        setMode(mode).setWeight(weight));
+        setMode(mode).setLimit(limit));
     
     RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
     CachePoolInfo info = iter.next().getInfo();
@@ -277,10 +303,10 @@ public class TestCacheDirectives {
     ownerName = "def";
     groupName = "456";
     mode = new FsPermission((short)0700);
-    weight = 151;
+    limit = 151;
     dfs.modifyCachePool(new CachePoolInfo(poolName).
         setOwnerName(ownerName).setGroupName(groupName).
-        setMode(mode).setWeight(weight));
+        setMode(mode).setLimit(limit));
 
     iter = dfs.listCachePools();
     info = iter.next().getInfo();
@@ -288,7 +314,7 @@ public class TestCacheDirectives {
     assertEquals(ownerName, info.getOwnerName());
     assertEquals(groupName, info.getGroupName());
     assertEquals(mode, info.getMode());
-    assertEquals(Integer.valueOf(weight), info.getWeight());
+    assertEquals(limit, (long)info.getLimit());
 
     dfs.removeCachePool(poolName);
     iter = dfs.listCachePools();
@@ -495,30 +521,22 @@ public class TestCacheDirectives {
 
   @Test(timeout=60000)
   public void testCacheManagerRestart() throws Exception {
-    cluster.shutdown();
-    cluster = null;
-    HdfsConfiguration conf = createCachingConf();
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
-
-    cluster.waitActive();
-    DistributedFileSystem dfs = cluster.getFileSystem();
-
     // Create and validate a pool
     final String pool = "poolparty";
     String groupName = "partygroup";
     FsPermission mode = new FsPermission((short)0777);
-    int weight = 747;
+    long limit = 747;
     dfs.addCachePool(new CachePoolInfo(pool)
         .setGroupName(groupName)
         .setMode(mode)
-        .setWeight(weight));
+        .setLimit(limit));
     RemoteIterator<CachePoolEntry> pit = dfs.listCachePools();
     assertTrue("No cache pools found", pit.hasNext());
     CachePoolInfo info = pit.next().getInfo();
     assertEquals(pool, info.getPoolName());
     assertEquals(groupName, info.getGroupName());
     assertEquals(mode, info.getMode());
-    assertEquals(weight, (int)info.getWeight());
+    assertEquals(limit, (long)info.getLimit());
     assertFalse("Unexpected # of cache pools found", pit.hasNext());
   
     // Create some cache entries
@@ -556,7 +574,7 @@ public class TestCacheDirectives {
     assertEquals(pool, info.getPoolName());
     assertEquals(groupName, info.getGroupName());
     assertEquals(mode, info.getMode());
-    assertEquals(weight, (int)info.getWeight());
+    assertEquals(limit, (long)info.getLimit());
     assertFalse("Unexpected # of cache pools found", pit.hasNext());
   
     dit = dfs.listCacheDirectives(null);
@@ -762,91 +780,64 @@ public class TestCacheDirectives {
         numCachedReplicas);
   }
 
-  private static final long BLOCK_SIZE = 512;
-  private static final int NUM_DATANODES = 4;
-
-  // Most Linux installs will allow non-root users to lock 64KB.
-  private static final long CACHE_CAPACITY = 64 * 1024 / NUM_DATANODES;
-
-  private static HdfsConfiguration createCachingConf() {
-    HdfsConfiguration conf = new HdfsConfiguration();
-    conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, CACHE_CAPACITY);
-    conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
-    conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, true);
-    conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000);
-    conf.setLong(DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1000);
-    return conf;
-  }
-
   @Test(timeout=120000)
   public void testWaitForCachedReplicas() throws Exception {
-    HdfsConfiguration conf = createCachingConf();
     FileSystemTestHelper helper = new FileSystemTestHelper();
-    MiniDFSCluster cluster =
-      new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
-
-    try {
-      cluster.waitActive();
-      DistributedFileSystem dfs = cluster.getFileSystem();
-      final NameNode namenode = cluster.getNameNode();
-      GenericTestUtils.waitFor(new Supplier<Boolean>() {
-        @Override
-        public Boolean get() {
-          return ((namenode.getNamesystem().getCacheCapacity() ==
-              (NUM_DATANODES * CACHE_CAPACITY)) &&
-                (namenode.getNamesystem().getCacheUsed() == 0));
-        }
-      }, 500, 60000);
-
-      NamenodeProtocols nnRpc = namenode.getRpcServer();
-      Path rootDir = helper.getDefaultWorkingDirectory(dfs);
-      // Create the pool
-      final String pool = "friendlyPool";
-      nnRpc.addCachePool(new CachePoolInfo("friendlyPool"));
-      // Create some test files
-      final int numFiles = 2;
-      final int numBlocksPerFile = 2;
-      final List<String> paths = new ArrayList<String>(numFiles);
-      for (int i=0; i<numFiles; i++) {
-        Path p = new Path(rootDir, "testCachePaths-" + i);
-        FileSystemTestHelper.createFile(dfs, p, numBlocksPerFile,
-            (int)BLOCK_SIZE);
-        paths.add(p.toUri().getPath());
-      }
-      // Check the initial statistics at the namenode
-      waitForCachedBlocks(namenode, 0, 0, "testWaitForCachedReplicas:0");
-      // Cache and check each path in sequence
-      int expected = 0;
-      for (int i=0; i<numFiles; i++) {
-        CacheDirectiveInfo directive =
-            new CacheDirectiveInfo.Builder().
-              setPath(new Path(paths.get(i))).
-              setPool(pool).
-              build();
-        nnRpc.addCacheDirective(directive);
-        expected += numBlocksPerFile;
-        waitForCachedBlocks(namenode, expected, expected,
-            "testWaitForCachedReplicas:1");
-      }
-      // Uncache and check each path in sequence
-      RemoteIterator<CacheDirectiveEntry> entries =
-        new CacheDirectiveIterator(nnRpc, null);
-      for (int i=0; i<numFiles; i++) {
-        CacheDirectiveEntry entry = entries.next();
-        nnRpc.removeCacheDirective(entry.getInfo().getId());
-        expected -= numBlocksPerFile;
-        waitForCachedBlocks(namenode, expected, expected,
-            "testWaitForCachedReplicas:2");
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return ((namenode.getNamesystem().getCacheCapacity() ==
+            (NUM_DATANODES * CACHE_CAPACITY)) &&
+              (namenode.getNamesystem().getCacheUsed() == 0));
       }
-    } finally {
-      cluster.shutdown();
+    }, 500, 60000);
+
+    NamenodeProtocols nnRpc = namenode.getRpcServer();
+    Path rootDir = helper.getDefaultWorkingDirectory(dfs);
+    // Create the pool
+    final String pool = "friendlyPool";
+    nnRpc.addCachePool(new CachePoolInfo("friendlyPool"));
+    // Create some test files
+    final int numFiles = 2;
+    final int numBlocksPerFile = 2;
+    final List<String> paths = new ArrayList<String>(numFiles);
+    for (int i=0; i<numFiles; i++) {
+      Path p = new Path(rootDir, "testCachePaths-" + i);
+      FileSystemTestHelper.createFile(dfs, p, numBlocksPerFile,
+          (int)BLOCK_SIZE);
+      paths.add(p.toUri().getPath());
+    }
+    // Check the initial statistics at the namenode
+    waitForCachedBlocks(namenode, 0, 0, "testWaitForCachedReplicas:0");
+    // Cache and check each path in sequence
+    int expected = 0;
+    for (int i=0; i<numFiles; i++) {
+      CacheDirectiveInfo directive =
+          new CacheDirectiveInfo.Builder().
+            setPath(new Path(paths.get(i))).
+            setPool(pool).
+            build();
+      nnRpc.addCacheDirective(directive, EnumSet.noneOf(CacheFlag.class));
+      expected += numBlocksPerFile;
+      waitForCachedBlocks(namenode, expected, expected,
+          "testWaitForCachedReplicas:1");
+    }
+    // Uncache and check each path in sequence
+    RemoteIterator<CacheDirectiveEntry> entries =
+      new CacheDirectiveIterator(nnRpc, null);
+    for (int i=0; i<numFiles; i++) {
+      CacheDirectiveEntry entry = entries.next();
+      nnRpc.removeCacheDirective(entry.getInfo().getId());
+      expected -= numBlocksPerFile;
+      waitForCachedBlocks(namenode, expected, expected,
+          "testWaitForCachedReplicas:2");
     }
   }
 
   @Test(timeout=120000)
   public void testAddingCacheDirectiveInfosWhenCachingIsDisabled()
       throws Exception {
+    cluster.shutdown();
     HdfsConfiguration conf = createCachingConf();
     conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, false);
     MiniDFSCluster cluster =
@@ -894,103 +885,92 @@ public class TestCacheDirectives {
 
   @Test(timeout=120000)
   public void testWaitForCachedReplicasInDirectory() throws Exception {
-    HdfsConfiguration conf = createCachingConf();
-    MiniDFSCluster cluster =
-      new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
-
-    try {
-      cluster.waitActive();
-      DistributedFileSystem dfs = cluster.getFileSystem();
-      NameNode namenode = cluster.getNameNode();
-      // Create the pool
-      final String pool = "friendlyPool";
-      final CachePoolInfo poolInfo = new CachePoolInfo(pool);
-      dfs.addCachePool(poolInfo);
-      // Create some test files
-      final List<Path> paths = new LinkedList<Path>();
-      paths.add(new Path("/foo/bar"));
-      paths.add(new Path("/foo/baz"));
-      paths.add(new Path("/foo2/bar2"));
-      paths.add(new Path("/foo2/baz2"));
-      dfs.mkdir(new Path("/foo"), FsPermission.getDirDefault());
-      dfs.mkdir(new Path("/foo2"), FsPermission.getDirDefault());
-      final int numBlocksPerFile = 2;
-      for (Path path : paths) {
-        FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
-            (int)BLOCK_SIZE, (short)3, false);
-      }
-      waitForCachedBlocks(namenode, 0, 0,
-          "testWaitForCachedReplicasInDirectory:0");
-
-      // cache entire directory
-      long id = dfs.addCacheDirective(
-            new CacheDirectiveInfo.Builder().
-              setPath(new Path("/foo")).
-              setReplication((short)2).
-              setPool(pool).
-              build());
-      waitForCachedBlocks(namenode, 4, 8,
-          "testWaitForCachedReplicasInDirectory:1:blocks");
-      // Verify that listDirectives gives the stats we want.
-      waitForCacheDirectiveStats(dfs,
-          4 * numBlocksPerFile * BLOCK_SIZE, 4 * numBlocksPerFile * BLOCK_SIZE,
-          2, 2,
-          new CacheDirectiveInfo.Builder().
-              setPath(new Path("/foo")).
-              build(),
-          "testWaitForCachedReplicasInDirectory:1:directive");
-      waitForCachePoolStats(dfs,
-          4 * numBlocksPerFile * BLOCK_SIZE, 4 * numBlocksPerFile * BLOCK_SIZE,
-          2, 2,
-          poolInfo, "testWaitForCachedReplicasInDirectory:1:pool");
+    // Create the pool
+    final String pool = "friendlyPool";
+    final CachePoolInfo poolInfo = new CachePoolInfo(pool);
+    dfs.addCachePool(poolInfo);
+    // Create some test files
+    final List<Path> paths = new LinkedList<Path>();
+    paths.add(new Path("/foo/bar"));
+    paths.add(new Path("/foo/baz"));
+    paths.add(new Path("/foo2/bar2"));
+    paths.add(new Path("/foo2/baz2"));
+    dfs.mkdir(new Path("/foo"), FsPermission.getDirDefault());
+    dfs.mkdir(new Path("/foo2"), FsPermission.getDirDefault());
+    final int numBlocksPerFile = 2;
+    for (Path path : paths) {
+      FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
+          (int)BLOCK_SIZE, (short)3, false);
+    }
+    waitForCachedBlocks(namenode, 0, 0,
+        "testWaitForCachedReplicasInDirectory:0");
 
-      long id2 = dfs.addCacheDirective(
-            new CacheDirectiveInfo.Builder().
-              setPath(new Path("/foo/bar")).
-              setReplication((short)4).
-              setPool(pool).
-              build());
-      // wait for an additional 2 cached replicas to come up
-      waitForCachedBlocks(namenode, 4, 10,
-          "testWaitForCachedReplicasInDirectory:2:blocks");
-      // the directory directive's stats are unchanged
-      waitForCacheDirectiveStats(dfs,
-          4 * numBlocksPerFile * BLOCK_SIZE, 4 * numBlocksPerFile * BLOCK_SIZE,
-          2, 2,
+    // cache entire directory
+    long id = dfs.addCacheDirective(
           new CacheDirectiveInfo.Builder().
-              setPath(new Path("/foo")).
-              build(),
-          "testWaitForCachedReplicasInDirectory:2:directive-1");
-      // verify /foo/bar's stats
-      waitForCacheDirectiveStats(dfs,
-          4 * numBlocksPerFile * BLOCK_SIZE,
-          // only 3 because the file only has 3 replicas, not 4 as requested.
-          3 * numBlocksPerFile * BLOCK_SIZE,
-          1,
-          // only 0 because the file can't be fully cached
-          0,
+            setPath(new Path("/foo")).
+            setReplication((short)2).
+            setPool(pool).
+            build());
+    waitForCachedBlocks(namenode, 4, 8,
+        "testWaitForCachedReplicasInDirectory:1:blocks");
+    // Verify that listDirectives gives the stats we want.
+    waitForCacheDirectiveStats(dfs,
+        4 * numBlocksPerFile * BLOCK_SIZE, 4 * numBlocksPerFile * BLOCK_SIZE,
+        2, 2,
+        new CacheDirectiveInfo.Builder().
+            setPath(new Path("/foo")).
+            build(),
+        "testWaitForCachedReplicasInDirectory:1:directive");
+    waitForCachePoolStats(dfs,
+        4 * numBlocksPerFile * BLOCK_SIZE, 4 * numBlocksPerFile * BLOCK_SIZE,
+        2, 2,
+        poolInfo, "testWaitForCachedReplicasInDirectory:1:pool");
+
+    long id2 = dfs.addCacheDirective(
           new CacheDirectiveInfo.Builder().
-              setPath(new Path("/foo/bar")).
-              build(),
-          "testWaitForCachedReplicasInDirectory:2:directive-2");
-      waitForCachePoolStats(dfs,
-          (4+4) * numBlocksPerFile * BLOCK_SIZE,
-          (4+3) * numBlocksPerFile * BLOCK_SIZE,
-          3, 2,
-          poolInfo, "testWaitForCachedReplicasInDirectory:2:pool");
-
-      // remove and watch numCached go to 0
-      dfs.removeCacheDirective(id);
-      dfs.removeCacheDirective(id2);
-      waitForCachedBlocks(namenode, 0, 0,
-          "testWaitForCachedReplicasInDirectory:3:blocks");
-      waitForCachePoolStats(dfs,
-          0, 0,
-          0, 0,
-          poolInfo, "testWaitForCachedReplicasInDirectory:3:pool");
-    } finally {
-      cluster.shutdown();
-    }
+            setPath(new Path("/foo/bar")).
+            setReplication((short)4).
+            setPool(pool).
+            build());
+    // wait for an additional 2 cached replicas to come up
+    waitForCachedBlocks(namenode, 4, 10,
+        "testWaitForCachedReplicasInDirectory:2:blocks");
+    // the directory directive's stats are unchanged
+    waitForCacheDirectiveStats(dfs,
+        4 * numBlocksPerFile * BLOCK_SIZE, 4 * numBlocksPerFile * BLOCK_SIZE,
+        2, 2,
+        new CacheDirectiveInfo.Builder().
+            setPath(new Path("/foo")).
+            build(),
+        "testWaitForCachedReplicasInDirectory:2:directive-1");
+    // verify /foo/bar's stats
+    waitForCacheDirectiveStats(dfs,
+        4 * numBlocksPerFile * BLOCK_SIZE,
+        // only 3 because the file only has 3 replicas, not 4 as requested.
+        3 * numBlocksPerFile * BLOCK_SIZE,
+        1,
+        // only 0 because the file can't be fully cached
+        0,
+        new CacheDirectiveInfo.Builder().
+            setPath(new Path("/foo/bar")).
+            build(),
+        "testWaitForCachedReplicasInDirectory:2:directive-2");
+    waitForCachePoolStats(dfs,
+        (4+4) * numBlocksPerFile * BLOCK_SIZE,
+        (4+3) * numBlocksPerFile * BLOCK_SIZE,
+        3, 2,
+        poolInfo, "testWaitForCachedReplicasInDirectory:2:pool");
+
+    // remove and watch numCached go to 0
+    dfs.removeCacheDirective(id);
+    dfs.removeCacheDirective(id2);
+    waitForCachedBlocks(namenode, 0, 0,
+        "testWaitForCachedReplicasInDirectory:3:blocks");
+    waitForCachePoolStats(dfs,
+        0, 0,
+        0, 0,
+        poolInfo, "testWaitForCachedReplicasInDirectory:3:pool");
   }
 
   /**
@@ -1000,68 +980,57 @@ public class TestCacheDirectives {
    */
   @Test(timeout=120000)
   public void testReplicationFactor() throws Exception {
-    HdfsConfiguration conf = createCachingConf();
-    MiniDFSCluster cluster =
-      new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
-
-    try {
-      cluster.waitActive();
-      DistributedFileSystem dfs = cluster.getFileSystem();
-      NameNode namenode = cluster.getNameNode();
-      // Create the pool
-      final String pool = "friendlyPool";
-      dfs.addCachePool(new CachePoolInfo(pool));
-      // Create some test files
-      final List<Path> paths = new LinkedList<Path>();
-      paths.add(new Path("/foo/bar"));
-      paths.add(new Path("/foo/baz"));
-      paths.add(new Path("/foo2/bar2"));
-      paths.add(new Path("/foo2/baz2"));
-      dfs.mkdir(new Path("/foo"), FsPermission.getDirDefault());
-      dfs.mkdir(new Path("/foo2"), FsPermission.getDirDefault());
-      final int numBlocksPerFile = 2;
-      for (Path path : paths) {
-        FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
-            (int)BLOCK_SIZE, (short)3, false);
-      }
-      waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:0");
-      checkNumCachedReplicas(dfs, paths, 0, 0);
-      // cache directory
-      long id = dfs.addCacheDirective(
+    // Create the pool
+    final String pool = "friendlyPool";
+    dfs.addCachePool(new CachePoolInfo(pool));
+    // Create some test files
+    final List<Path> paths = new LinkedList<Path>();
+    paths.add(new Path("/foo/bar"));
+    paths.add(new Path("/foo/baz"));
+    paths.add(new Path("/foo2/bar2"));
+    paths.add(new Path("/foo2/baz2"));
+    dfs.mkdir(new Path("/foo"), FsPermission.getDirDefault());
+    dfs.mkdir(new Path("/foo2"), FsPermission.getDirDefault());
+    final int numBlocksPerFile = 2;
+    for (Path path : paths) {
+      FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
+          (int)BLOCK_SIZE, (short)3, false);
+    }
+    waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:0");
+    checkNumCachedReplicas(dfs, paths, 0, 0);
+    // cache directory
+    long id = dfs.addCacheDirective(
+        new CacheDirectiveInfo.Builder().
+          setPath(new Path("/foo")).
+          setReplication((short)1).
+          setPool(pool).
+          build());
+    waitForCachedBlocks(namenode, 4, 4, "testReplicationFactor:1");
+    checkNumCachedReplicas(dfs, paths, 4, 4);
+    // step up the replication factor
+    for (int i=2; i<=3; i++) {
+      dfs.modifyCacheDirective(
           new CacheDirectiveInfo.Builder().
-            setPath(new Path("/foo")).
-            setReplication((short)1).
-            setPool(pool).
-            build());
-      waitForCachedBlocks(namenode, 4, 4, "testReplicationFactor:1");
-      checkNumCachedReplicas(dfs, paths, 4, 4);
-      // step up the replication factor
-      for (int i=2; i<=3; i++) {
-        dfs.modifyCacheDirective(
-            new CacheDirectiveInfo.Builder().
-            setId(id).
-            setReplication((short)i).
-            build());
-        waitForCachedBlocks(namenode, 4, 4*i, "testReplicationFactor:2");
-        checkNumCachedReplicas(dfs, paths, 4, 4*i);
-      }
-      // step it down
-      for (int i=2; i>=1; i--) {
-        dfs.modifyCacheDirective(
-            new CacheDirectiveInfo.Builder().
-            setId(id).
-            setReplication((short)i).
-            build());
-        waitForCachedBlocks(namenode, 4, 4*i, "testReplicationFactor:3");
-        checkNumCachedReplicas(dfs, paths, 4, 4*i);
-      }
-      // remove and watch numCached go to 0
-      dfs.removeCacheDirective(id);
-      waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:4");
-      checkNumCachedReplicas(dfs, paths, 0, 0);
-    } finally {
-      cluster.shutdown();
+          setId(id).
+          setReplication((short)i).
+          build());
+      waitForCachedBlocks(namenode, 4, 4*i, "testReplicationFactor:2");
+      checkNumCachedReplicas(dfs, paths, 4, 4*i);
+    }
+    // step it down
+    for (int i=2; i>=1; i--) {
+      dfs.modifyCacheDirective(
+          new CacheDirectiveInfo.Builder().
+          setId(id).
+          setReplication((short)i).
+          build());
+      waitForCachedBlocks(namenode, 4, 4*i, "testReplicationFactor:3");
+      checkNumCachedReplicas(dfs, paths, 4, 4*i);
     }
+    // remove and watch numCached go to 0
+    dfs.removeCacheDirective(id);
+    waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:4");
+    checkNumCachedReplicas(dfs, paths, 0, 0);
   }
 
   @Test(timeout=60000)
@@ -1081,11 +1050,12 @@ public class TestCacheDirectives {
     assertNull("Unexpected owner name", info.getOwnerName());
     assertNull("Unexpected group name", info.getGroupName());
     assertNull("Unexpected mode", info.getMode());
-    assertNull("Unexpected weight", info.getWeight());
+    assertNull("Unexpected limit", info.getLimit());
     // Modify the pool so myuser is now the owner
+    final long limit = 99;
     dfs.modifyCachePool(new CachePoolInfo(poolName)
         .setOwnerName(myUser.getShortUserName())
-        .setWeight(99));
+        .setLimit(limit));
     // Should see full info
     it = myDfs.listCachePools();
     info = it.next().getInfo();
@@ -1096,60 +1066,127 @@ public class TestCacheDirectives {
     assertNotNull("Expected group name", info.getGroupName());
     assertEquals("Mismatched mode", (short) 0700,
         info.getMode().toShort());
-    assertEquals("Mismatched weight", 99, (int)info.getWeight());
+    assertEquals("Mismatched limit", limit, (long)info.getLimit());
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=120000)
   public void testExpiry() throws Exception {
-    HdfsConfiguration conf = createCachingConf();
-    MiniDFSCluster cluster =
-      new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
+    String pool = "pool1";
+    dfs.addCachePool(new CachePoolInfo(pool));
+    Path p = new Path("/mypath");
+    DFSTestUtil.createFile(dfs, p, BLOCK_SIZE*2, (short)2, 0x999);
+    // Expire after test timeout
+    Date start = new Date();
+    Date expiry = DateUtils.addSeconds(start, 120);
+    final long id = dfs.addCacheDirective(new CacheDirectiveInfo.Builder()
+        .setPath(p)
+        .setPool(pool)
+        .setExpiration(CacheDirectiveInfo.Expiration.newAbsolute(expiry))
+        .setReplication((short)2)
+        .build());
+    waitForCachedBlocks(cluster.getNameNode(), 2, 4, "testExpiry:1");
+    // Change it to expire sooner
+    dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder().setId(id)
+        .setExpiration(Expiration.newRelative(0)).build());
+    waitForCachedBlocks(cluster.getNameNode(), 0, 0, "testExpiry:2");
+    RemoteIterator<CacheDirectiveEntry> it = dfs.listCacheDirectives(null);
+    CacheDirectiveEntry ent = it.next();
+    assertFalse(it.hasNext());
+    Date entryExpiry = new Date(ent.getInfo().getExpiration().getMillis());
+    assertTrue("Directive should have expired",
+        entryExpiry.before(new Date()));
+    // Change it back to expire later
+    dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder().setId(id)
+        .setExpiration(Expiration.newRelative(120000)).build());
+    waitForCachedBlocks(cluster.getNameNode(), 2, 4, "testExpiry:3");
+    it = dfs.listCacheDirectives(null);
+    ent = it.next();
+    assertFalse(it.hasNext());
+    entryExpiry = new Date(ent.getInfo().getExpiration().getMillis());
+    assertTrue("Directive should not have expired",
+        entryExpiry.after(new Date()));
+    // Verify that setting a negative TTL throws an error
     try {
-      DistributedFileSystem dfs = cluster.getFileSystem();
-      String pool = "pool1";
-      dfs.addCachePool(new CachePoolInfo(pool));
-      Path p = new Path("/mypath");
-      DFSTestUtil.createFile(dfs, p, BLOCK_SIZE*2, (short)2, 0x999);
-      // Expire after test timeout
-      Date start = new Date();
-      Date expiry = DateUtils.addSeconds(start, 120);
-      final long id = dfs.addCacheDirective(new CacheDirectiveInfo.Builder()
-          .setPath(p)
-          .setPool(pool)
-          .setExpiration(CacheDirectiveInfo.Expiration.newAbsolute(expiry))
-          .setReplication((short)2)
-          .build());
-      waitForCachedBlocks(cluster.getNameNode(), 2, 4, "testExpiry:1");
-      // Change it to expire sooner
       dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder().setId(id)
-          .setExpiration(Expiration.newRelative(0)).build());
-      waitForCachedBlocks(cluster.getNameNode(), 0, 0, "testExpiry:2");
-      RemoteIterator<CacheDirectiveEntry> it = dfs.listCacheDirectives(null);
-      CacheDirectiveEntry ent = it.next();
-      assertFalse(it.hasNext());
-      Date entryExpiry = new Date(ent.getInfo().getExpiration().getMillis());
-      assertTrue("Directive should have expired",
-          entryExpiry.before(new Date()));
-      // Change it back to expire later
-      dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder().setId(id)
-          .setExpiration(Expiration.newRelative(120000)).build());
-      waitForCachedBlocks(cluster.getNameNode(), 2, 4, "testExpiry:3");
-      it = dfs.listCacheDirectives(null);
-      ent = it.next();
-      assertFalse(it.hasNext());
-      entryExpiry = new Date(ent.getInfo().getExpiration().getMillis());
-      assertTrue("Directive should not have expired",
-          entryExpiry.after(new Date()));
-      // Verify that setting a negative TTL throws an error
-      try {
-        dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder().setId(id)
-            .setExpiration(Expiration.newRelative(-1)).build());
-      } catch (InvalidRequestException e) {
-        GenericTestUtils
-            .assertExceptionContains("Cannot set a negative expiration", e);
-      }
-    } finally {
-      cluster.shutdown();
+          .setExpiration(Expiration.newRelative(-1)).build());
+    } catch (InvalidRequestException e) {
+      GenericTestUtils
+          .assertExceptionContains("Cannot set a negative expiration", e);
+    }
+  }
+
+  @Test(timeout=120000)
+  public void testLimit() throws Exception {
+    try {
+      dfs.addCachePool(new CachePoolInfo("poolofnegativity").setLimit(-99l));
+      fail("Should not be able to set a negative limit");
+    } catch (InvalidRequestException e) {
+      GenericTestUtils.assertExceptionContains("negative", e);
+    }
+    final String destiny = "poolofdestiny";
+    final Path path1 = new Path("/destiny");
+    DFSTestUtil.createFile(dfs, path1, 2*BLOCK_SIZE, (short)1, 0x9494);
+    // Start off with a limit that is too small
+    final CachePoolInfo poolInfo = new CachePoolInfo(destiny)
+        .setLimit(2*BLOCK_SIZE-1);
+    dfs.addCachePool(poolInfo);
+    final CacheDirectiveInfo info1 = new CacheDirectiveInfo.Builder()
+        .setPool(destiny).setPath(path1).build();
+    try {
+      dfs.addCacheDirective(info1);
+      fail("Should not be able to cache when there is no more limit");
+    } catch (InvalidRequestException e) {
+      GenericTestUtils.assertExceptionContains("remaining capacity", e);
+    }
+    // Raise the limit up to fit and it should work this time
+    poolInfo.setLimit(2*BLOCK_SIZE);
+    dfs.modifyCachePool(poolInfo);
+    long id1 = dfs.addCacheDirective(info1);
+    waitForCachePoolStats(dfs,
+        2*BLOCK_SIZE, 2*BLOCK_SIZE,
+        1, 1,
+        poolInfo, "testLimit:1");
+    // Adding another file, it shouldn't be cached
+    final Path path2 = new Path("/failure");
+    DFSTestUtil.createFile(dfs, path2, BLOCK_SIZE, (short)1, 0x9495);
+    try {
+      dfs.addCacheDirective(new CacheDirectiveInfo.Builder()
+          .setPool(destiny).setPath(path2).build(),
+          EnumSet.noneOf(CacheFlag.class));
+      fail("Should not be able to add another cached file");
+    } catch (InvalidRequestException e) {
+      GenericTestUtils.assertExceptionContains("remaining capacity", e);
+    }
+    // Bring the limit down, the first file should get uncached
+    poolInfo.setLimit(BLOCK_SIZE);
+    dfs.modifyCachePool(poolInfo);
+    waitForCachePoolStats(dfs,
+        2*BLOCK_SIZE, 0,
+        1, 0,
+        poolInfo, "testLimit:2");
+    RemoteIterator<CachePoolEntry> it = dfs.listCachePools();
+    assertTrue("Expected a cache pool", it.hasNext());
+    CachePoolStats stats = it.next().getStats();
+    assertEquals("Overlimit bytes should be difference of needed and limit",
+        BLOCK_SIZE, stats.getBytesOverlimit());
+    // Moving a directive to a pool without enough limit should fail
+    CachePoolInfo inadequate =
+        new CachePoolInfo("poolofinadequacy").setLimit(BLOCK_SIZE);
+    dfs.addCachePool(inadequate);
+    try {
+      dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(info1)
+          .setId(id1).setPool(inadequate.getPoolName()).build(),
+          EnumSet.noneOf(CacheFlag.class));
+    } catch(InvalidRequestException e) {
+      GenericTestUtils.assertExceptionContains("remaining capacity", e);
     }
+    // Succeeds when force=true
+    dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(info1).setId(id1)
+        .setPool(inadequate.getPoolName()).build(),
+        EnumSet.of(CacheFlag.FORCE));
+    // Also can add with force=true
+    dfs.addCacheDirective(
+        new CacheDirectiveInfo.Builder().setPool(inadequate.getPoolName())
+            .setPath(path1).build(), EnumSet.of(CacheFlag.FORCE));
   }
 }

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -20,6 +20,9 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil.assertNNHasCheckpoints;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil.getNameNodeCurrentDirs;
+import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt;
+import static org.apache.hadoop.test.MetricsAsserts.assertGaugeGt;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -72,6 +75,7 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
@@ -106,6 +110,7 @@ public class TestCheckpoint {
   }
 
   static final Log LOG = LogFactory.getLog(TestCheckpoint.class); 
+  static final String NN_METRICS = "NameNodeActivity";
   
   static final long seed = 0xDEADBEEFL;
   static final int blockSize = 4096;
@@ -1048,6 +1053,14 @@ public class TestCheckpoint {
       //
       secondary = startSecondaryNameNode(conf);
       secondary.doCheckpoint();
+
+      MetricsRecordBuilder rb = getMetrics(NN_METRICS);
+      assertCounterGt("GetImageNumOps", 0, rb);
+      assertCounterGt("GetEditNumOps", 0, rb);
+      assertCounterGt("PutImageNumOps", 0, rb);
+      assertGaugeGt("GetImageAvgTime", 0.0, rb);
+      assertGaugeGt("GetEditAvgTime", 0.0, rb);
+      assertGaugeGt("PutImageAvgTime", 0.0, rb);
     } finally {
       fileSys.close();
       cleanup(secondary);

+ 30 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

@@ -31,6 +31,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileChannel;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.SortedMap;
 
@@ -383,4 +384,33 @@ public class TestFSEditLogLoader {
     assertTrue(!validation.hasCorruptHeader());
     assertEquals(HdfsConstants.INVALID_TXID, validation.getEndTxId());
   }
+
+  private static final Map<Byte, FSEditLogOpCodes> byteToEnum =
+      new HashMap<Byte, FSEditLogOpCodes>();
+  static {
+    for(FSEditLogOpCodes opCode : FSEditLogOpCodes.values()) {
+      byteToEnum.put(opCode.getOpCode(), opCode);
+    }
+  }
+
+  private static FSEditLogOpCodes fromByte(byte opCode) {
+    return byteToEnum.get(opCode);
+  }
+
+  @Test
+  public void testFSEditLogOpCodes() throws IOException {
+    //try all codes
+    for(FSEditLogOpCodes c : FSEditLogOpCodes.values()) {
+      final byte code = c.getOpCode();
+      assertEquals("c=" + c + ", code=" + code,
+          c, FSEditLogOpCodes.fromByte(code));
+    }
+
+    //try all byte values
+    for(int b = 0; b < (1 << Byte.SIZE); b++) {
+      final byte code = (byte)b;
+      assertEquals("b=" + b + ", code=" + code,
+          fromByte(code), FSEditLogOpCodes.fromByte(code));
+    }
+  }
 }

+ 9 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java

@@ -38,6 +38,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -760,7 +761,7 @@ public class TestRetryCacheWithHA {
 
     @Override
     void invoke() throws Exception {
-      result = client.addCacheDirective(directive);
+      result = client.addCacheDirective(directive, EnumSet.of(CacheFlag.FORCE));
     }
 
     @Override
@@ -802,7 +803,7 @@ public class TestRetryCacheWithHA {
     @Override
     void prepare() throws Exception {
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
-      id = client.addCacheDirective(directive);
+      id = client.addCacheDirective(directive, EnumSet.of(CacheFlag.FORCE));
     }
 
     @Override
@@ -811,7 +812,7 @@ public class TestRetryCacheWithHA {
           new CacheDirectiveInfo.Builder().
               setId(id).
               setReplication(newReplication).
-              build());
+              build(), EnumSet.of(CacheFlag.FORCE));
     }
 
     @Override
@@ -858,7 +859,7 @@ public class TestRetryCacheWithHA {
     @Override
     void prepare() throws Exception {
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
-      id = dfs.addCacheDirective(directive);
+      id = dfs.addCacheDirective(directive, EnumSet.of(CacheFlag.FORCE));
     }
 
     @Override
@@ -936,19 +937,19 @@ public class TestRetryCacheWithHA {
 
     @Override
     void prepare() throws Exception {
-      client.addCachePool(new CachePoolInfo(pool).setWeight(10));
+      client.addCachePool(new CachePoolInfo(pool).setLimit(10l));
     }
 
     @Override
     void invoke() throws Exception {
-      client.modifyCachePool(new CachePoolInfo(pool).setWeight(99));
+      client.modifyCachePool(new CachePoolInfo(pool).setLimit(99l));
     }
 
     @Override
     boolean checkNamenodeBeforeReturn() throws Exception {
       for (int i = 0; i < CHECKTIMES; i++) {
         RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
-        if (iter.hasNext() && iter.next().getInfo().getWeight() == 99) {
+        if (iter.hasNext() && (long)iter.next().getInfo().getLimit() == 99) {
           return true;
         }
         Thread.sleep(1000);
@@ -1216,7 +1217,7 @@ public class TestRetryCacheWithHA {
       CacheDirectiveInfo directiveInfo =
         new CacheDirectiveInfo.Builder().setPool(poolName).setPath(path).build();
       dfs.addCachePool(new CachePoolInfo(poolName));
-      dfs.addCacheDirective(directiveInfo);
+      dfs.addCacheDirective(directiveInfo, EnumSet.of(CacheFlag.FORCE));
       poolNames.add(poolName);
     }
     listCacheDirectives(poolNames, 0);

二进制
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


+ 92 - 94
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -13,8 +13,8 @@
       <TXID>2</TXID>
       <DELEGATION_KEY>
         <KEY_ID>1</KEY_ID>
-        <EXPIRY_DATE>1386695013416</EXPIRY_DATE>
-        <KEY>360a10c6ecac725e</KEY>
+        <EXPIRY_DATE>1387701670577</EXPIRY_DATE>
+        <KEY>7bb5467995769b59</KEY>
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -24,8 +24,8 @@
       <TXID>3</TXID>
       <DELEGATION_KEY>
         <KEY_ID>2</KEY_ID>
-        <EXPIRY_DATE>1386695013425</EXPIRY_DATE>
-        <KEY>9b110c0b83225f7d</KEY>
+        <EXPIRY_DATE>1387701670580</EXPIRY_DATE>
+        <KEY>a5a3a2755e36827b</KEY>
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -37,17 +37,17 @@
       <INODEID>16386</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814612</MTIME>
-      <ATIME>1386003814612</ATIME>
+      <MTIME>1387010471220</MTIME>
+      <ATIME>1387010471220</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>7</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -59,13 +59,13 @@
       <INODEID>0</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814665</MTIME>
-      <ATIME>1386003814612</ATIME>
+      <MTIME>1387010471276</MTIME>
+      <ATIME>1387010471220</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -78,8 +78,8 @@
       <LENGTH>0</LENGTH>
       <SRC>/file_create_u\0001;F431</SRC>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1386003814671</TIMESTAMP>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <TIMESTAMP>1387010471286</TIMESTAMP>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>9</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -89,8 +89,8 @@
       <TXID>7</TXID>
       <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
-      <TIMESTAMP>1386003814678</TIMESTAMP>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <TIMESTAMP>1387010471299</TIMESTAMP>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>10</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -101,9 +101,9 @@
       <LENGTH>0</LENGTH>
       <INODEID>16387</INODEID>
       <PATH>/directory_mkdir</PATH>
-      <TIMESTAMP>1386003814686</TIMESTAMP>
+      <TIMESTAMP>1387010471312</TIMESTAMP>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>493</MODE>
       </PERMISSION_STATUS>
@@ -136,7 +136,7 @@
       <TXID>12</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>15</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -147,7 +147,7 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>16</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -157,7 +157,7 @@
       <TXID>14</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>17</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -169,17 +169,17 @@
       <INODEID>16388</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814712</MTIME>
-      <ATIME>1386003814712</ATIME>
+      <MTIME>1387010471373</MTIME>
+      <ATIME>1387010471373</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>18</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -191,13 +191,13 @@
       <INODEID>0</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814714</MTIME>
-      <ATIME>1386003814712</ATIME>
+      <MTIME>1387010471380</MTIME>
+      <ATIME>1387010471373</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -253,9 +253,9 @@
       <LENGTH>0</LENGTH>
       <SRC>/file_create_u\0001;F431</SRC>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1386003814732</TIMESTAMP>
+      <TIMESTAMP>1387010471428</TIMESTAMP>
       <OPTIONS>NONE</OPTIONS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>25</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -267,17 +267,17 @@
       <INODEID>16389</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814737</MTIME>
-      <ATIME>1386003814737</ATIME>
+      <MTIME>1387010471438</MTIME>
+      <ATIME>1387010471438</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>27</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -388,8 +388,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814889</MTIME>
-      <ATIME>1386003814737</ATIME>
+      <MTIME>1387010471540</MTIME>
+      <ATIME>1387010471438</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -409,7 +409,7 @@
         <GENSTAMP>1003</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -423,17 +423,17 @@
       <INODEID>16390</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814891</MTIME>
-      <ATIME>1386003814891</ATIME>
+      <MTIME>1387010471547</MTIME>
+      <ATIME>1387010471547</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>40</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -544,8 +544,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814914</MTIME>
-      <ATIME>1386003814891</ATIME>
+      <MTIME>1387010471588</MTIME>
+      <ATIME>1387010471547</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -565,7 +565,7 @@
         <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -579,17 +579,17 @@
       <INODEID>16391</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814916</MTIME>
-      <ATIME>1386003814916</ATIME>
+      <MTIME>1387010471595</MTIME>
+      <ATIME>1387010471595</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>52</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -700,8 +700,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814938</MTIME>
-      <ATIME>1386003814916</ATIME>
+      <MTIME>1387010471651</MTIME>
+      <ATIME>1387010471595</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -721,7 +721,7 @@
         <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -733,12 +733,12 @@
       <TXID>56</TXID>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
-      <TIMESTAMP>1386003814940</TIMESTAMP>
+      <TIMESTAMP>1387010471663</TIMESTAMP>
       <SOURCES>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE2>/file_concat_1</SOURCE2>
       </SOURCES>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>63</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -750,14 +750,14 @@
       <INODEID>16392</INODEID>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
-      <MTIME>1386003814956</MTIME>
-      <ATIME>1386003814956</ATIME>
+      <MTIME>1387010471674</MTIME>
+      <ATIME>1387010471674</ATIME>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>511</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>64</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -768,14 +768,14 @@
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
-        <OWNER>aagarwal</OWNER>
+        <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
-        <ISSUE_DATE>1386003814961</ISSUE_DATE>
-        <MAX_DATE>1386608614961</MAX_DATE>
+        <ISSUE_DATE>1387010471682</ISSUE_DATE>
+        <MAX_DATE>1387615271682</MAX_DATE>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
-      <EXPIRY_TIME>1386090214961</EXPIRY_TIME>
+      <EXPIRY_TIME>1387096871682</EXPIRY_TIME>
     </DATA>
   </RECORD>
   <RECORD>
@@ -785,14 +785,14 @@
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
-        <OWNER>aagarwal</OWNER>
+        <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
-        <ISSUE_DATE>1386003814961</ISSUE_DATE>
-        <MAX_DATE>1386608614961</MAX_DATE>
+        <ISSUE_DATE>1387010471682</ISSUE_DATE>
+        <MAX_DATE>1387615271682</MAX_DATE>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
-      <EXPIRY_TIME>1386090215078</EXPIRY_TIME>
+      <EXPIRY_TIME>1387096871717</EXPIRY_TIME>
     </DATA>
   </RECORD>
   <RECORD>
@@ -802,11 +802,11 @@
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
-        <OWNER>aagarwal</OWNER>
+        <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
-        <ISSUE_DATE>1386003814961</ISSUE_DATE>
-        <MAX_DATE>1386608614961</MAX_DATE>
+        <ISSUE_DATE>1387010471682</ISSUE_DATE>
+        <MAX_DATE>1387615271682</MAX_DATE>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
     </DATA>
@@ -816,13 +816,11 @@
     <DATA>
       <TXID>61</TXID>
       <POOLNAME>poolparty</POOLNAME>
-      <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
-        <GROUPNAME>staff</GROUPNAME>
-        <MODE>493</MODE>
-      </PERMISSION_STATUS>
-      <WEIGHT>100</WEIGHT>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <OWNERNAME>andrew</OWNERNAME>
+      <GROUPNAME>andrew</GROUPNAME>
+      <MODE>493</MODE>
+      <LIMIT>9223372036854775807</LIMIT>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>68</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -834,8 +832,8 @@
       <OWNERNAME>carlton</OWNERNAME>
       <GROUPNAME>party</GROUPNAME>
       <MODE>448</MODE>
-      <WEIGHT>1989</WEIGHT>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <LIMIT>1989</LIMIT>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>69</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -848,7 +846,7 @@
       <REPLICATION>1</REPLICATION>
       <POOL>poolparty</POOL>
       <EXPIRATION>-1</EXPIRATION>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>70</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -858,7 +856,7 @@
       <TXID>64</TXID>
       <ID>1</ID>
       <PATH>/bar2</PATH>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>71</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -867,7 +865,7 @@
     <DATA>
       <TXID>65</TXID>
       <ID>1</ID>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>72</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -876,7 +874,7 @@
     <DATA>
       <TXID>66</TXID>
       <POOLNAME>poolparty</POOLNAME>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>73</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -888,17 +886,17 @@
       <INODEID>16393</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003815135</MTIME>
-      <ATIME>1386003815135</ATIME>
+      <MTIME>1387010471802</MTIME>
+      <ATIME>1387010471802</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>74</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -955,7 +953,7 @@
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
       <TXID>73</TXID>
-      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-1253204429_1</LEASEHOLDER>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-52011019_1</LEASEHOLDER>
       <PATH>/hard-lease-recovery-test</PATH>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
     </DATA>
@@ -968,8 +966,8 @@
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003817462</MTIME>
-      <ATIME>1386003815135</ATIME>
+      <MTIME>1387010474126</MTIME>
+      <ATIME>1387010471802</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -979,7 +977,7 @@
         <GENSTAMP>1011</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml

@@ -80,8 +80,8 @@
     <test> <!--Tested -->
       <description>Testing modifying a cache pool</description>
       <test-commands>
-        <cache-admin-command>-addPool poolparty -owner alice -group alicegroup -mode 0000 -weight 50</cache-admin-command>
-        <cache-admin-command>-modifyPool poolparty -owner bob -group bobgroup -mode 0777 -weight 51</cache-admin-command>
+        <cache-admin-command>-addPool poolparty -owner alice -group alicegroup -mode 0000 -limit 50</cache-admin-command>
+        <cache-admin-command>-modifyPool poolparty -owner bob -group bobgroup -mode 0777 -limit 51</cache-admin-command>
         <cache-admin-command>-listPools</cache-admin-command>
       </test-commands>
       <cleanup-commands>
@@ -90,7 +90,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>poolparty  bob    bobgroup  rwxrwxrwx       51</expected-output>
+          <expected-output>poolparty  bob    bobgroup  rwxrwxrwx      51</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -129,11 +129,11 @@
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>bar   alice  alicegroup  rwxr-xr-x      100</expected-output>
+          <expected-output>bar   alice  alicegroup  rwxr-xr-x   unlimited</expected-output>
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>foo   bob    bob         rw-rw-r--      100</expected-output>
+          <expected-output>foo   bob    bob         rw-rw-r--   unlimited</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -156,7 +156,7 @@
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>foo   bob    bob    rw-rw-r--      100</expected-output>
+          <expected-output>foo   bob    bob    rw-rw-r--   unlimited</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -417,11 +417,11 @@
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>bar   alice  alicegroup  rwxr-xr-x      100             0             0             0             0</expected-output>
+          <expected-output>bar   alice  alicegroup  rwxr-xr-x   unlimited             0             0                0             0             0</expected-output>
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>foo   bob    bob         rw-rw-r--      100             0             0             0             0</expected-output>
+          <expected-output>foo   bob    bob         rw-rw-r--   unlimited             0             0                0             0             0</expected-output>
         </comparator>
       </comparators>
     </test>

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

@@ -71,6 +71,12 @@ Trunk (Unreleased)
     MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing.
     (Srikanth Sundarrajan via amareshwari)
 
+    MAPREDUCE-5197. Add a service for checkpointing task state.
+    (Carlo Curino via cdouglas)
+
+    MAPREDUCE-5189. Add policies and wiring to respond to preemption requests
+    from YARN. (Carlo Curino via cdouglas)
+
   BUG FIXES
 
     MAPREDUCE-4272. SortedRanges.Range#compareTo is not spec compliant.
@@ -240,6 +246,12 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5623. TestJobCleanup fails because of RejectedExecutionException
     and NPE. (jlowe)
 
+    MAPREDUCE-5679. TestJobHistoryParsing has race condition (Liyin Liang via
+    jlowe)
+
+    MAPREDUCE-5687. Fixed failure in TestYARNRunner caused by YARN-1446. (Jian He
+    via vinodkv)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 5 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.PolicyProvider;
@@ -84,14 +85,17 @@ public class TaskAttemptListenerImpl extends CompositeService
       .newSetFromMap(new ConcurrentHashMap<WrappedJvmID, Boolean>()); 
   
   private JobTokenSecretManager jobTokenSecretManager = null;
+  private AMPreemptionPolicy preemptionPolicy;
   
   public TaskAttemptListenerImpl(AppContext context,
       JobTokenSecretManager jobTokenSecretManager,
-      RMHeartbeatHandler rmHeartbeatHandler) {
+      RMHeartbeatHandler rmHeartbeatHandler,
+      AMPreemptionPolicy preemptionPolicy) {
     super(TaskAttemptListenerImpl.class.getName());
     this.context = context;
     this.jobTokenSecretManager = jobTokenSecretManager;
     this.rmHeartbeatHandler = rmHeartbeatHandler;
+    this.preemptionPolicy = preemptionPolicy;
   }
 
   @Override

+ 20 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -102,6 +102,8 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerRequestor;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.NoopAMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
@@ -188,8 +190,8 @@ public class MRAppMaster extends CompositeService {
   private ContainerLauncher containerLauncher;
   private EventHandler<CommitterEvent> committerEventHandler;
   private Speculator speculator;
-  private TaskAttemptListener taskAttemptListener;
-  private JobTokenSecretManager jobTokenSecretManager =
+  protected TaskAttemptListener taskAttemptListener;
+  protected JobTokenSecretManager jobTokenSecretManager =
       new JobTokenSecretManager();
   private JobId jobId;
   private boolean newApiCommitter;
@@ -197,6 +199,7 @@ public class MRAppMaster extends CompositeService {
   private JobEventDispatcher jobEventDispatcher;
   private JobHistoryEventHandler jobHistoryEventHandler;
   private SpeculatorEventDispatcher speculatorEventDispatcher;
+  private AMPreemptionPolicy preemptionPolicy;
 
   private Job job;
   private Credentials jobCredentials = new Credentials(); // Filled during init
@@ -383,8 +386,12 @@ public class MRAppMaster extends CompositeService {
       committerEventHandler = createCommitterEventHandler(context, committer);
       addIfService(committerEventHandler);
 
+      //policy handling preemption requests from RM
+      preemptionPolicy = createPreemptionPolicy(conf);
+      preemptionPolicy.init(context);
+
       //service to handle requests to TaskUmbilicalProtocol
-      taskAttemptListener = createTaskAttemptListener(context);
+      taskAttemptListener = createTaskAttemptListener(context, preemptionPolicy);
       addIfService(taskAttemptListener);
 
       //service to log job history events
@@ -475,6 +482,12 @@ public class MRAppMaster extends CompositeService {
     return committer;
   }
 
+  protected AMPreemptionPolicy createPreemptionPolicy(Configuration conf) {
+    return ReflectionUtils.newInstance(conf.getClass(
+          MRJobConfig.MR_AM_PREEMPTION_POLICY,
+          NoopAMPreemptionPolicy.class, AMPreemptionPolicy.class), conf);
+  }
+
   protected boolean keepJobFiles(JobConf conf) {
     return (conf.getKeepTaskFilesPattern() != null || conf
         .getKeepFailedTaskFiles());
@@ -692,10 +705,11 @@ public class MRAppMaster extends CompositeService {
     }
   }
 
-  protected TaskAttemptListener createTaskAttemptListener(AppContext context) {
+  protected TaskAttemptListener createTaskAttemptListener(AppContext context,
+      AMPreemptionPolicy preemptionPolicy) {
     TaskAttemptListener lis =
         new TaskAttemptListenerImpl(context, jobTokenSecretManager,
-            getRMHeartbeatHandler());
+            getRMHeartbeatHandler(), preemptionPolicy);
     return lis;
   }
 
@@ -805,7 +819,7 @@ public class MRAppMaster extends CompositeService {
             , containerID);
       } else {
         this.containerAllocator = new RMContainerAllocator(
-            this.clientService, this.context);
+            this.clientService, this.context, preemptionPolicy);
       }
       ((Service)this.containerAllocator).init(getConfig());
       ((Service)this.containerAllocator).start();

+ 47 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -57,6 +57,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdate
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -67,6 +68,7 @@ import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.client.api.NMTokenCache;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -147,13 +149,17 @@ public class RMContainerAllocator extends RMContainerRequestor
   private long retryInterval;
   private long retrystartTime;
 
+  private final AMPreemptionPolicy preemptionPolicy;
+
   BlockingQueue<ContainerAllocatorEvent> eventQueue
     = new LinkedBlockingQueue<ContainerAllocatorEvent>();
 
   private ScheduleStats scheduleStats = new ScheduleStats();
 
-  public RMContainerAllocator(ClientService clientService, AppContext context) {
+  public RMContainerAllocator(ClientService clientService, AppContext context,
+      AMPreemptionPolicy preemptionPolicy) {
     super(clientService, context);
+    this.preemptionPolicy = preemptionPolicy;
     this.stopped = new AtomicBoolean(false);
   }
 
@@ -361,11 +367,15 @@ public class RMContainerAllocator extends RMContainerRequestor
         LOG.error("Could not deallocate container for task attemptId " + 
             aId);
       }
+      preemptionPolicy.handleCompletedContainer(event.getAttemptID());
     } else if (
         event.getType() == ContainerAllocator.EventType.CONTAINER_FAILED) {
       ContainerFailedEvent fEv = (ContainerFailedEvent) event;
       String host = getHost(fEv.getContMgrAddress());
       containerFailedOnHost(host);
+      // propagate failures to preemption policy to discard checkpoints for
+      // failed tasks
+      preemptionPolicy.handleFailedContainer(event.getAttemptID());
     }
   }
 
@@ -399,7 +409,7 @@ public class RMContainerAllocator extends RMContainerRequestor
         }
         scheduledRequests.reduces.clear();
         
-        //preempt for making space for atleast one map
+        //preempt for making space for at least one map
         int premeptionLimit = Math.max(mapResourceReqt, 
             (int) (maxReducePreemptionLimit * memLimit));
         
@@ -409,7 +419,7 @@ public class RMContainerAllocator extends RMContainerRequestor
         int toPreempt = (int) Math.ceil((float) preemptMem/reduceResourceReqt);
         toPreempt = Math.min(toPreempt, assignedRequests.reduces.size());
         
-        LOG.info("Going to preempt " + toPreempt);
+        LOG.info("Going to preempt " + toPreempt + " due to lack of space for maps");
         assignedRequests.preemptReduce(toPreempt);
       }
     }
@@ -595,6 +605,14 @@ public class RMContainerAllocator extends RMContainerRequestor
     }
     
     List<ContainerStatus> finishedContainers = response.getCompletedContainersStatuses();
+
+    // propagate preemption requests
+    final PreemptionMessage preemptReq = response.getPreemptionMessage();
+    if (preemptReq != null) {
+      preemptionPolicy.preempt(
+          new PreemptionContext(assignedRequests), preemptReq);
+    }
+
     if (newContainers.size() + finishedContainers.size() > 0 || headRoom != newHeadRoom) {
       //something changed
       recalculateReduceSchedule = true;
@@ -630,7 +648,9 @@ public class RMContainerAllocator extends RMContainerRequestor
         String diagnostics = StringInterner.weakIntern(cont.getDiagnostics());
         eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptID,
             diagnostics));
-      }      
+
+        preemptionPolicy.handleCompletedContainer(attemptID);
+      }
     }
     return newContainers;
   }
@@ -1232,4 +1252,27 @@ public class RMContainerAllocator extends RMContainerRequestor
         " RackLocal:" + rackLocalAssigned);
     }
   }
+
+  static class PreemptionContext extends AMPreemptionPolicy.Context {
+    final AssignedRequests reqs;
+
+    PreemptionContext(AssignedRequests reqs) {
+      this.reqs = reqs;
+    }
+    @Override
+    public TaskAttemptId getTaskAttempt(ContainerId container) {
+      return reqs.get(container);
+    }
+
+    @Override
+    public List<Container> getContainers(TaskType t){
+      if(TaskType.REDUCE.equals(t))
+        return new ArrayList<Container>(reqs.reduces.values());
+      if(TaskType.MAP.equals(t))
+        return new ArrayList<Container>(reqs.maps.values());
+      return null;
+    }
+
+  }
+
 }

+ 117 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/preemption/AMPreemptionPolicy.java

@@ -0,0 +1,117 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.hadoop.mapreduce.v2.app.rm.preemption;
+
+import java.util.List;
+
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
+import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
+
+/**
+ * Policy encoding the {@link org.apache.hadoop.mapreduce.v2.app.MRAppMaster}
+ * response to preemption requests from the ResourceManager.
+ * @see org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator
+ */
+public interface AMPreemptionPolicy {
+
+  public abstract class Context {
+
+    /**
+     * @param container ID of container to preempt
+     * @return Task associated with the running container or <code>null</code>
+     * if no task is bound to that container.
+     */
+    public abstract TaskAttemptId getTaskAttempt(ContainerId container);
+
+    /**
+     * Method provides the complete list of containers running task of type t
+     * for this AM.
+     * @param t the type of containers
+     * @return a map containing
+     */
+    public abstract List<Container> getContainers(TaskType t);
+
+  }
+
+  public void init(AppContext context);
+
+  /**
+   * Callback informing the policy of ResourceManager. requests for resources
+   * to return to the cluster. The policy may take arbitrary action to satisfy
+   * requests by checkpointing task state, returning containers, or ignoring
+   * requests. The RM may elect to enforce these requests by forcibly killing
+   * containers not returned after some duration.
+   * @param context Handle to the current state of running containers
+   * @param preemptionRequests Request from RM for resources to return.
+   */
+  public void preempt(Context context, PreemptionMessage preemptionRequests);
+
+  /**
+   * This method is invoked by components interested to learn whether a certain
+   * task is being preempted.
+   * @param attemptID Task attempt to query
+   * @return true if this attempt is being preempted
+   */
+  public boolean isPreempted(TaskAttemptId attemptID);
+
+  /**
+   * This method is used to report to the policy that a certain task has been
+   * successfully preempted (for bookeeping, counters, etc..)
+   * @param attemptID Task attempt that preempted
+   */
+  public void reportSuccessfulPreemption(TaskAttemptID attemptID);
+
+  /**
+   * Callback informing the policy of containers exiting with a failure. This
+   * allows the policy to implemnt cleanup/compensating actions.
+   * @param attemptID Task attempt that failed
+   */
+  public void handleFailedContainer(TaskAttemptId attemptID);
+
+  /**
+   * Callback informing the policy of containers exiting cleanly. This is
+   * reported to the policy for bookeeping purposes.
+   * @param attemptID Task attempt that completed
+   */
+  public void handleCompletedContainer(TaskAttemptId attemptID);
+
+  /**
+   * Method to retrieve the latest checkpoint for a given {@link TaskID}
+   * @param taskId TaskID
+   * @return CheckpointID associated with this task or null
+   */
+  public TaskCheckpointID getCheckpointID(TaskID taskId);
+
+  /**
+   * Method to store the latest {@link
+   * org.apache.hadoop.mapreduce.checkpoint.CheckpointID} for a given {@link
+   * TaskID}. Assigning a null is akin to remove all previous checkpoints for
+   * this task.
+   * @param taskId TaskID
+   * @param cid Checkpoint to assign or <tt>null</tt> to remove it.
+   */
+  public void setCheckpointID(TaskID taskId, TaskCheckpointID cid);
+
+}

+ 111 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/preemption/KillAMPreemptionPolicy.java

@@ -0,0 +1,111 @@
+/**
+* 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.mapreduce.v2.app.rm.preemption;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
+import org.apache.hadoop.mapreduce.JobCounter;
+import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.PreemptionContainer;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
+import org.apache.hadoop.yarn.event.EventHandler;
+
+/**
+ * Sample policy that aggressively kills tasks when requested.
+ */
+public class KillAMPreemptionPolicy implements AMPreemptionPolicy {
+
+  private static final Log LOG =
+      LogFactory.getLog(KillAMPreemptionPolicy.class);
+
+  @SuppressWarnings("rawtypes")
+  private EventHandler dispatcher = null;
+
+  @Override
+  public void init(AppContext context) {
+    dispatcher = context.getEventHandler();
+  }
+
+  @Override
+  public void preempt(Context ctxt, PreemptionMessage preemptionRequests) {
+    // for both strict and negotiable preemption requests kill the
+    // container
+    for (PreemptionContainer c :
+        preemptionRequests.getStrictContract().getContainers()) {
+      killContainer(ctxt, c);
+    }
+    for (PreemptionContainer c :
+         preemptionRequests.getContract().getContainers()) {
+       killContainer(ctxt, c);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private void killContainer(Context ctxt, PreemptionContainer c){
+    ContainerId reqCont = c.getId();
+    TaskAttemptId reqTask = ctxt.getTaskAttempt(reqCont);
+    LOG.info("Evicting " + reqTask);
+    dispatcher.handle(new TaskAttemptEvent(reqTask,
+        TaskAttemptEventType.TA_KILL));
+
+    // add preemption to counters
+    JobCounterUpdateEvent jce = new JobCounterUpdateEvent(reqTask
+            .getTaskId().getJobId());
+        jce.addCounterUpdate(JobCounter.TASKS_REQ_PREEMPT, 1);
+        dispatcher.handle(jce);
+  }
+
+  @Override
+  public void handleFailedContainer(TaskAttemptId attemptID) {
+    // ignore
+  }
+
+  @Override
+  public boolean isPreempted(TaskAttemptId yarnAttemptID) {
+    return false;
+  }
+
+  @Override
+  public void reportSuccessfulPreemption(TaskAttemptID taskAttemptID) {
+    // ignore
+  }
+
+  @Override
+  public TaskCheckpointID getCheckpointID(TaskID taskId) {
+    return null;
+  }
+
+  @Override
+  public void setCheckpointID(TaskID taskId, TaskCheckpointID cid) {
+    // ignore
+  }
+
+  @Override
+  public void handleCompletedContainer(TaskAttemptId attemptID) {
+    // ignore
+  }
+
+}

+ 72 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/preemption/NoopAMPreemptionPolicy.java

@@ -0,0 +1,72 @@
+/**
+* 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.mapreduce.v2.app.rm.preemption;
+
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
+import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
+
+/**
+ * NoOp policy that ignores all the requests for preemption.
+ */
+public class NoopAMPreemptionPolicy implements AMPreemptionPolicy {
+
+  @Override
+  public void init(AppContext context){
+   // do nothing
+  }
+
+  @Override
+  public void preempt(Context ctxt, PreemptionMessage preemptionRequests) {
+    // do nothing, ignore all requeusts
+  }
+
+  @Override
+  public void handleFailedContainer(TaskAttemptId attemptID) {
+    // do nothing
+  }
+
+  @Override
+  public boolean isPreempted(TaskAttemptId yarnAttemptID) {
+    return false;
+  }
+
+  @Override
+  public void reportSuccessfulPreemption(TaskAttemptID taskAttemptID) {
+    // ignore
+  }
+
+  @Override
+  public TaskCheckpointID getCheckpointID(TaskID taskId) {
+    return null;
+  }
+
+  @Override
+  public void setCheckpointID(TaskID taskId, TaskCheckpointID cid) {
+    // ignore
+  }
+
+  @Override
+  public void handleCompletedContainer(TaskAttemptId attemptID) {
+    // ignore
+  }
+
+}

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java

@@ -60,7 +60,7 @@ public class TestTaskAttemptListenerImpl {
         JobTokenSecretManager jobTokenSecretManager,
         RMHeartbeatHandler rmHeartbeatHandler,
         TaskHeartbeatHandler hbHandler) {
-      super(context, jobTokenSecretManager, rmHeartbeatHandler);
+      super(context, jobTokenSecretManager, rmHeartbeatHandler, null);
       this.taskHeartbeatHandler = hbHandler;
     }
     
@@ -191,7 +191,7 @@ public class TestTaskAttemptListenerImpl {
         mock(RMHeartbeatHandler.class);
     final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
     TaskAttemptListenerImpl listener =
-        new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler) {
+        new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler, null) {
       @Override
       protected void registerHeartbeatHandler(Configuration conf) {
         taskHeartbeatHandler = hbHandler;
@@ -245,7 +245,7 @@ public class TestTaskAttemptListenerImpl {
         mock(RMHeartbeatHandler.class);
     final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
     TaskAttemptListenerImpl listener =
-        new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler) {
+        new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler, null) {
       @Override
       protected void registerHeartbeatHandler(Configuration conf) {
         taskHeartbeatHandler = hbHandler;

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java

@@ -79,6 +79,7 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetUtils;
@@ -467,7 +468,8 @@ public class MRApp extends MRAppMaster {
   }
 
   @Override
-  protected TaskAttemptListener createTaskAttemptListener(AppContext context) {
+  protected TaskAttemptListener createTaskAttemptListener(
+      AppContext context, AMPreemptionPolicy policy) {
     return new TaskAttemptListener(){
       @Override
       public InetSocketAddress getAddress() {

+ 8 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java

@@ -33,6 +33,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssigned
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.NoopAMPreemptionPolicy;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@@ -61,6 +63,8 @@ public class MRAppBenchmark {
 
   /**
    * Runs memory and time benchmark with Mock MRApp.
+   * @param app Application to submit
+   * @throws Exception On application failure
    */
   public void run(MRApp app) throws Exception {
     Logger rootLogger = LogManager.getRootLogger();
@@ -133,6 +137,7 @@ public class MRAppBenchmark {
       protected void serviceStart() throws Exception {
         thread = new Thread(new Runnable() {
           @Override
+          @SuppressWarnings("unchecked")
           public void run() {
             ContainerAllocatorEvent event = null;
             while (!Thread.currentThread().isInterrupted()) {
@@ -192,7 +197,9 @@ public class MRAppBenchmark {
       @Override
       protected ContainerAllocator createContainerAllocator(
           ClientService clientService, AppContext context) {
-        return new RMContainerAllocator(clientService, context) {
+
+        AMPreemptionPolicy policy = new NoopAMPreemptionPolicy();
+        return new RMContainerAllocator(clientService, context, policy) {
           @Override
           protected ApplicationMasterProtocol createSchedulerProxy() {
             return new ApplicationMasterProtocol() {

+ 4 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -247,13 +248,14 @@ public class TestFail {
       super(maps, reduces, false, "TimeOutTaskMRApp", true);
     }
     @Override
-    protected TaskAttemptListener createTaskAttemptListener(AppContext context) {
+    protected TaskAttemptListener createTaskAttemptListener(
+        AppContext context, AMPreemptionPolicy policy) {
       //This will create the TaskAttemptListener with TaskHeartbeatHandler
       //RPC servers are not started
       //task time out is reduced
       //when attempt times out, heartbeat handler will send the lost event
       //leading to Attempt failure
-      return new TaskAttemptListenerImpl(getContext(), null, null) {
+      return new TaskAttemptListenerImpl(getContext(), null, null, policy) {
         @Override
         public void startRpcServer(){};
         @Override

+ 11 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.NoopAMPreemptionPolicy;
+
 import static org.mockito.Matchers.anyFloat;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.isA;
@@ -1428,14 +1430,15 @@ public class TestRMContainerAllocator {
     // Use this constructor when using a real job.
     MyContainerAllocator(MyResourceManager rm,
         ApplicationAttemptId appAttemptId, AppContext context) {
-      super(createMockClientService(), context);
+      super(createMockClientService(), context, new NoopAMPreemptionPolicy());
       this.rm = rm;
     }
 
     // Use this constructor when you are using a mocked job.
     public MyContainerAllocator(MyResourceManager rm, Configuration conf,
         ApplicationAttemptId appAttemptId, Job job) {
-      super(createMockClientService(), createAppContext(appAttemptId, job));
+      super(createMockClientService(), createAppContext(appAttemptId, job),
+          new NoopAMPreemptionPolicy());
       this.rm = rm;
       super.init(conf);
       super.start();
@@ -1444,7 +1447,8 @@ public class TestRMContainerAllocator {
     public MyContainerAllocator(MyResourceManager rm, Configuration conf,
         ApplicationAttemptId appAttemptId, Job job, Clock clock) {
       super(createMockClientService(),
-          createAppContext(appAttemptId, job, clock));
+          createAppContext(appAttemptId, job, clock),
+          new NoopAMPreemptionPolicy());
       this.rm = rm;
       super.init(conf);
       super.start();
@@ -1671,7 +1675,8 @@ public class TestRMContainerAllocator {
         ApplicationId.newInstance(1, 1));
 
     RMContainerAllocator allocator = new RMContainerAllocator(
-        mock(ClientService.class), appContext) {
+        mock(ClientService.class), appContext,
+        new NoopAMPreemptionPolicy()) {
           @Override
           protected void register() {
           }
@@ -1721,7 +1726,8 @@ public class TestRMContainerAllocator {
   @Test
   public void testCompletedContainerEvent() {
     RMContainerAllocator allocator = new RMContainerAllocator(
-        mock(ClientService.class), mock(AppContext.class));
+        mock(ClientService.class), mock(AppContext.class),
+        new NoopAMPreemptionPolicy());
     
     TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(
         MRBuilderUtils.newTaskId(

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

@@ -45,5 +45,9 @@ public enum JobCounter {
   TOTAL_LAUNCHED_UBERTASKS,
   NUM_UBER_SUBMAPS,
   NUM_UBER_SUBREDUCES,
-  NUM_FAILED_UBERTASKS
+  NUM_FAILED_UBERTASKS,
+  TASKS_REQ_PREEMPT,
+  CHECKPOINTS,
+  CHECKPOINT_BYTES,
+  CHECKPOINT_TIME
 }

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

@@ -459,7 +459,13 @@ public interface MRJobConfig {
   public static final String MR_AM_JOB_REDUCE_PREEMPTION_LIMIT = 
     MR_AM_PREFIX  + "job.reduce.preemption.limit";
   public static final float DEFAULT_MR_AM_JOB_REDUCE_PREEMPTION_LIMIT = 0.5f;
-  
+
+  /**
+   * Policy class encoding responses to preemption requests.
+   */
+  public static final String MR_AM_PREEMPTION_POLICY =
+    MR_AM_PREFIX + "preemption.policy";
+
   /** AM ACL disabled. **/
   public static final String JOB_AM_ACCESS_DISABLED = 
     "mapreduce.job.am-access-disabled";
@@ -708,4 +714,7 @@ public interface MRJobConfig {
   
   public static final String MR_APPLICATION_TYPE = "MAPREDUCE";
   
+  public static final String TASK_PREEMPTION =
+      "mapreduce.job.preemption";
+
 }

+ 30 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/CheckpointID.java

@@ -0,0 +1,30 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * This class represent the identified (memento) for a checkpoint. It is allowed
+ * to contain small amount of metadata about a checkpoint and must provide
+ * sufficient information to the corresponding CheckpointService to locate and
+ * retrieve the data contained in the checkpoint.
+ */
+public interface CheckpointID extends Writable {
+
+}

+ 31 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/CheckpointNamingService.java

@@ -0,0 +1,31 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+/**
+ * This class represent a naming service for checkpoints.
+ */
+public interface CheckpointNamingService {
+
+  /**
+   * Generate a new checkpoint Name
+   * @return the checkpoint name
+   */
+  public String getNewName();
+
+}

+ 100 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/CheckpointService.java

@@ -0,0 +1,100 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+import java.io.IOException;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+
+/**
+ * The CheckpointService provides a simple API to store and retrieve the state of a task.
+ *
+ * Checkpoints are atomic, single-writer, write-once, multiple-readers,
+ * ready-many type of objects. This is provided by releasing the CheckpointID
+ * for a checkpoint only upon commit of the checkpoint, and by preventing a
+ * checkpoint to be re-opened for writes.
+ *
+ * Non-functional properties such as durability, availability, compression,
+ * garbage collection, quotas are left to the implementation.
+ *
+ * This API is envisioned as the basic building block for a checkpoint service,
+ * on top of which richer interfaces can be layered (e.g., frameworks providing
+ * object-serialization, checkpoint metadata and provenance, etc.)
+ *
+ */
+public interface CheckpointService {
+
+  public interface CheckpointWriteChannel extends WritableByteChannel { }
+  public interface CheckpointReadChannel extends ReadableByteChannel { }
+
+  /**
+   * This method creates a checkpoint and provide a channel to write to it. The
+   * name/location of the checkpoint are unknown to the user as of this time, in
+   * fact, the CheckpointID is not released to the user until commit is called.
+   * This makes enforcing atomicity of writes easy.
+   * @return a channel that can be used to write to the checkpoint
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public CheckpointWriteChannel create()
+    throws IOException, InterruptedException;
+
+  /**
+   * Used to finalize and existing checkpoint. It returns the CheckpointID that
+   * can be later used to access (read-only) this checkpoint. This guarantees
+   * atomicity of the checkpoint.
+   * @param ch the CheckpointWriteChannel to commit
+   * @return a CheckpointID
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public CheckpointID commit(CheckpointWriteChannel ch)
+    throws IOException, InterruptedException;
+
+  /**
+   * Dual to commit, it aborts the current checkpoint. Garbage collection
+   * choices are left to the implementation. The CheckpointID is not generated
+   * nor released to the user so the checkpoint is not accessible.
+   * @param ch the CheckpointWriteChannel to abort
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void abort(CheckpointWriteChannel ch)
+      throws IOException, InterruptedException;
+
+  /**
+   * Given a CheckpointID returns a reading channel.
+   * @param id CheckpointID for the checkpoint to be opened
+   * @return a CheckpointReadChannel
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public CheckpointReadChannel open(CheckpointID id)
+    throws IOException, InterruptedException;
+
+  /**
+   * It discards an existing checkpoint identified by its CheckpointID.
+   * @param  id CheckpointID for the checkpoint to be deleted
+   * @return a boolean confirming success of the deletion
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public boolean delete(CheckpointID id)
+    throws IOException, InterruptedException;
+
+}

+ 26 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/EnumCounter.java

@@ -0,0 +1,26 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+public enum EnumCounter {
+  INPUTKEY,
+  INPUTVALUE,
+  OUTPUTRECORDS,
+  CHECKPOINT_BYTES,
+  CHECKPOINT_MS
+}

+ 72 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/FSCheckpointID.java

@@ -0,0 +1,72 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+
+/**
+ * A FileSystem based checkpoint ID contains reference to the Path
+ * where the checkpoint has been saved.
+ */
+public class FSCheckpointID implements CheckpointID {
+
+  private Path path;
+
+  public FSCheckpointID(){
+  }
+
+  public FSCheckpointID(Path path) {
+    this.path = path;
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  @Override
+  public String toString() {
+    return path.toString();
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, path.toString());
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    this.path = new Path(Text.readString(in));
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return other instanceof FSCheckpointID
+      && path.equals(((FSCheckpointID)other).path);
+  }
+
+  @Override
+  public int hashCode() {
+    return path.hashCode();
+  }
+
+}

+ 193 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/FSCheckpointService.java

@@ -0,0 +1,193 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A FileSystem based CheckpointService.
+ */
+public class FSCheckpointService implements CheckpointService {
+
+  private final Path base;
+  private final FileSystem fs;
+  private final CheckpointNamingService namingPolicy;
+  private final short replication;
+
+  public FSCheckpointService(FileSystem fs, Path base,
+      CheckpointNamingService namingPolicy, short replication) {
+    this.fs = fs;
+    this.base = base;
+    this.namingPolicy = namingPolicy;
+    this.replication = replication;
+  }
+
+  public CheckpointWriteChannel create()
+      throws IOException {
+
+    String name = namingPolicy.getNewName();
+
+    Path p = new Path(name);
+    if (p.isUriPathAbsolute()) {
+      throw new IOException("Checkpoint cannot be an absolute path");
+    }
+    return createInternal(new Path(base, p));
+  }
+
+  CheckpointWriteChannel createInternal(Path name) throws IOException {
+
+    //create a temp file, fail if file exists
+    return new FSCheckpointWriteChannel(name, fs.create(tmpfile(name),
+          replication));
+  }
+
+  private static class FSCheckpointWriteChannel
+      implements CheckpointWriteChannel {
+    private boolean isOpen = true;
+    private final Path finalDst;
+    private final WritableByteChannel out;
+
+    FSCheckpointWriteChannel(Path finalDst, FSDataOutputStream out) {
+      this.finalDst = finalDst;
+      this.out = Channels.newChannel(out);
+    }
+
+    public int write(ByteBuffer b) throws IOException {
+      return out.write(b);
+    }
+
+    public Path getDestination() {
+      return finalDst;
+    }
+
+    @Override
+    public void close() throws IOException {
+      isOpen=false;
+      out.close();
+    }
+
+    @Override
+    public boolean isOpen() {
+      return isOpen;
+    }
+
+  }
+
+  @Override
+  public CheckpointReadChannel open(CheckpointID id)
+      throws IOException, InterruptedException {
+      if (!(id instanceof FSCheckpointID)) {
+        throw new IllegalArgumentException(
+            "Mismatched checkpoint type: " + id.getClass());
+      }
+      return new FSCheckpointReadChannel(
+          fs.open(((FSCheckpointID) id).getPath()));
+  }
+
+  private static class FSCheckpointReadChannel
+      implements CheckpointReadChannel {
+
+    private boolean isOpen = true;
+    private final ReadableByteChannel in;
+
+    FSCheckpointReadChannel(FSDataInputStream in){
+      this.in = Channels.newChannel(in);
+    }
+
+    @Override
+    public int read(ByteBuffer bb) throws IOException {
+      return in.read(bb);
+    }
+
+    @Override
+    public void close() throws IOException {
+      isOpen = false;
+      in.close();
+    }
+
+    @Override
+    public boolean isOpen() {
+      return isOpen;
+    }
+
+  }
+
+  @Override
+  public CheckpointID commit(CheckpointWriteChannel ch)
+      throws IOException, InterruptedException {
+    if (ch.isOpen()) {
+      ch.close();
+    }
+    FSCheckpointWriteChannel hch = (FSCheckpointWriteChannel)ch;
+    Path dst = hch.getDestination();
+    if (!fs.rename(tmpfile(dst), dst)) {
+      // attempt to clean up
+      abort(ch);
+      throw new IOException("Failed to promote checkpoint" +
+      		 tmpfile(dst) + " -> " + dst);
+    }
+    return new FSCheckpointID(hch.getDestination());
+  }
+
+  @Override
+  public void abort(CheckpointWriteChannel ch) throws IOException {
+    if (ch.isOpen()) {
+      ch.close();
+    }
+    FSCheckpointWriteChannel hch = (FSCheckpointWriteChannel)ch;
+    Path tmp = tmpfile(hch.getDestination());
+    try {
+      if (!fs.delete(tmp, false)) {
+        throw new IOException("Failed to delete checkpoint during abort");
+      }
+    } catch (FileNotFoundException e) {
+      // IGNORE
+    }
+  }
+
+  @Override
+  public boolean delete(CheckpointID id) throws IOException,
+      InterruptedException {
+    if (!(id instanceof FSCheckpointID)) {
+      throw new IllegalArgumentException(
+          "Mismatched checkpoint type: " + id.getClass());
+    }
+    Path tmp = ((FSCheckpointID)id).getPath();
+    try {
+      return fs.delete(tmp, false);
+    } catch (FileNotFoundException e) {
+      // IGNORE
+    }
+    return true;
+  }
+
+  static final Path tmpfile(Path p) {
+    return new Path(p.getParent(), p.getName() + ".tmp");
+  }
+
+}

+ 32 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/RandomNameCNS.java

@@ -0,0 +1,32 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+import org.apache.commons.lang.RandomStringUtils;
+
+/**
+ * Simple naming service that generates a random checkpoint name.
+ */
+public class RandomNameCNS implements CheckpointNamingService {
+
+  @Override
+  public String getNewName() {
+    return "checkpoint_" + RandomStringUtils.randomAlphanumeric(8);
+  }
+
+}

+ 39 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/SimpleNamingService.java

@@ -0,0 +1,39 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+/**
+ * A naming service that simply returns the name it has been initialized with.
+ */
+public class SimpleNamingService implements CheckpointNamingService{
+
+  final String name;
+
+  public SimpleNamingService(String name){
+    this.name = name;
+  }
+
+  /**
+   * Generate a new checkpoint Name
+   * @return the checkpoint name
+   */
+  public String getNewName(){
+    return "checkpoint_" + name;
+  }
+
+}

+ 126 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/checkpoint/TaskCheckpointID.java

@@ -0,0 +1,126 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapred.Counters;
+
+/**
+ * Implementation of CheckpointID used in MR. It contains a reference to an
+ * underlying FileSsytem based checkpoint, and various metadata about the
+ * cost of checkpoints and other counters. This is sent by the task to the AM
+ * to be stored and provided to the next execution of the same task.
+ */
+public class TaskCheckpointID implements CheckpointID{
+
+  FSCheckpointID rawId;
+  private List<Path> partialOutput;
+  private Counters counters;
+
+  public TaskCheckpointID() {
+    this.rawId = new FSCheckpointID();
+    this.partialOutput = new ArrayList<Path>();
+  }
+
+  public TaskCheckpointID(FSCheckpointID rawId, List<Path> partialOutput,
+          Counters counters) {
+    this.rawId = rawId;
+    this.counters = counters;
+    if(partialOutput == null)
+      this.partialOutput = new ArrayList<Path>();
+    else
+      this.partialOutput = partialOutput;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    counters.write(out);
+    if (partialOutput == null) {
+      WritableUtils.writeVLong(out, 0L);
+    } else {
+      WritableUtils.writeVLong(out, partialOutput.size());
+      for(Path p:partialOutput){
+        Text.writeString(out, p.toString());
+      }
+    }
+    rawId.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    partialOutput.clear();
+    counters.readFields(in);
+    long numPout = WritableUtils.readVLong(in);
+    for(int i=0;i<numPout;i++)
+      partialOutput.add(new Path(Text.readString(in)));
+    rawId.readFields(in);
+  }
+
+  @Override
+  public boolean equals(Object other){
+    if (other instanceof TaskCheckpointID){
+      return this.rawId.equals(((TaskCheckpointID)other).rawId) &&
+             this.counters.equals(((TaskCheckpointID) other).counters) &&
+             this.partialOutput.containsAll(((TaskCheckpointID) other).partialOutput) &&
+             ((TaskCheckpointID) other).partialOutput.containsAll(this.partialOutput);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return rawId.hashCode();
+  }
+
+  /**
+   * @return the size of the checkpoint in bytes
+   */
+  public long getCheckpointBytes() {
+    return counters.findCounter(EnumCounter.CHECKPOINT_BYTES).getValue();
+  }
+
+  /**
+   * @return how long it took to take this checkpoint
+   */
+  public long getCheckpointTime() {
+    return counters.findCounter(EnumCounter.CHECKPOINT_MS).getValue();
+  }
+
+  public String toString(){
+    return rawId.toString() + " counters:" + counters;
+
+  }
+
+  public List<Path> getPartialCommittedOutput() {
+    return partialOutput;
+  }
+
+  public Counters getCounters() {
+    return counters;
+  }
+
+}

+ 4 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/org/apache/hadoop/mapreduce/JobCounter.properties

@@ -27,3 +27,7 @@ SLOTS_MILLIS_MAPS.name=            Total time spent by all maps in occupied slot
 SLOTS_MILLIS_REDUCES.name=         Total time spent by all reduces in occupied slots (ms)
 FALLOW_SLOTS_MILLIS_MAPS.name=     Total time spent by all maps waiting after reserving slots (ms)
 FALLOW_SLOTS_MILLIS_REDUCES.name=  Total time spent by all reduces waiting after reserving slots (ms)
+TASKS_REQ_PREEMPT.name=            Tasks that have been asked to preempt
+CHECKPOINTS.name=                  Number of checkpoints reported
+CHECKPOINT_BYTES.name=             Total amount of bytes in checkpoints
+CHECKPOINT_TIME.name=              Total time spent checkpointing (ms)

+ 48 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/checkpoint/TestFSCheckpointID.java

@@ -0,0 +1,48 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.junit.Test;
+
+public class TestFSCheckpointID {
+
+  @Test
+  public void testFSCheckpointIDSerialization() throws IOException {
+
+    Path inpath = new Path("/tmp/blah");
+    FSCheckpointID cidin = new FSCheckpointID(inpath);
+    DataOutputBuffer out = new DataOutputBuffer();
+    cidin.write(out);
+    out.close();
+
+    FSCheckpointID cidout = new FSCheckpointID(null);
+    DataInputBuffer in = new DataInputBuffer();
+    in.reset(out.getData(), 0, out.getLength());
+    cidout.readFields(in);
+    in.close();
+
+    assert cidin.equals(cidout);
+
+  }
+
+}

+ 102 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/checkpoint/TestFSCheckpointService.java

@@ -0,0 +1,102 @@
+/**
+ * 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.mapreduce.checkpoint;
+
+import java.nio.ByteBuffer;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.mapreduce.checkpoint.CheckpointService.CheckpointWriteChannel;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+import org.mockito.*;
+
+public class TestFSCheckpointService {
+
+  private final int BUFSIZE = 1024;
+
+  @Test
+  public void testCheckpointCreate() throws Exception {
+    checkpointCreate(ByteBuffer.allocate(BUFSIZE));
+  }
+
+  @Test
+  public void testCheckpointCreateDirect() throws Exception {
+    checkpointCreate(ByteBuffer.allocateDirect(BUFSIZE));
+  }
+
+  public void checkpointCreate(ByteBuffer b) throws Exception {
+    int WRITES = 128;
+    FileSystem fs = mock(FileSystem.class);
+    DataOutputBuffer dob = new DataOutputBuffer();
+    FSDataOutputStream hdfs = spy(new FSDataOutputStream(dob, null));
+    @SuppressWarnings("resource") // backed by array
+    DataOutputBuffer verif = new DataOutputBuffer();
+    when(fs.create(isA(Path.class), eq((short)1))).thenReturn(hdfs);
+    when(fs.rename(isA(Path.class), isA(Path.class))).thenReturn(true);
+
+    Path base = new Path("/chk");
+    Path finalLoc = new Path("/chk/checkpoint_chk0");
+    Path tmp = FSCheckpointService.tmpfile(finalLoc);
+
+    FSCheckpointService chk = new FSCheckpointService(fs, base,
+        new SimpleNamingService("chk0"), (short) 1);
+    CheckpointWriteChannel out = chk.create();
+
+    Random r = new Random();
+
+    final byte[] randBytes = new byte[BUFSIZE];
+    for (int i = 0; i < WRITES; ++i) {
+      r.nextBytes(randBytes);
+      int s = r.nextInt(BUFSIZE - 1);
+      int e = r.nextInt(BUFSIZE - s) + 1;
+      verif.write(randBytes, s, e);
+      b.clear();
+      b.put(randBytes).flip();
+      b.position(s).limit(b.position() + e);
+      out.write(b);
+    }
+    verify(fs, never()).rename(any(Path.class), eq(finalLoc));
+    CheckpointID cid = chk.commit(out);
+    verify(hdfs).close();
+    verify(fs).rename(eq(tmp), eq(finalLoc));
+
+    assertArrayEquals(Arrays.copyOfRange(verif.getData(), 0, verif.getLength()),
+        Arrays.copyOfRange(dob.getData(), 0, dob.getLength()));
+  }
+
+  @Test
+  public void testDelete() throws Exception {
+    FileSystem fs = mock(FileSystem.class);
+    Path chkloc = new Path("/chk/chk0");
+    when(fs.delete(eq(chkloc), eq(false))).thenReturn(true);
+    Path base = new Path("/otherchk");
+    FSCheckpointID id = new FSCheckpointID(chkloc);
+    FSCheckpointService chk = new FSCheckpointService(fs, base,
+        new SimpleNamingService("chk0"), (short) 1);
+    assertTrue(chk.delete(id));
+    verify(fs).delete(eq(chkloc), eq(false));
+  }
+
+}

+ 39 - 39
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java

@@ -339,8 +339,11 @@ public class TestJobHistoryParsing {
     PrintStream stdps = System.out;
     try {
       System.setOut(new PrintStream(outContent));
-      HistoryViewer viewer = new HistoryViewer(fc.makeQualified(
-          fileInfo.getHistoryFile()).toString(), conf, true);
+      HistoryViewer viewer;
+      synchronized (fileInfo) {
+        viewer = new HistoryViewer(fc.makeQualified(
+            fileInfo.getHistoryFile()).toString(), conf, true);
+      }
       viewer.print();
 
       for (TaskInfo taskInfo : allTasks.values()) {
@@ -397,29 +400,27 @@ public class TestJobHistoryParsing {
       // make sure all events are flushed
       app.waitForState(Service.STATE.STOPPED);
 
-      String jobhistoryDir = JobHistoryUtils
-          .getHistoryIntermediateDoneDirForUser(conf);
       JobHistory jobHistory = new JobHistory();
       jobHistory.init(conf);
+      HistoryFileInfo fileInfo = jobHistory.getJobFileInfo(jobId);
+      
+      JobHistoryParser parser;
+      JobInfo jobInfo;
+      synchronized (fileInfo) {
+        Path historyFilePath = fileInfo.getHistoryFile();
+        FSDataInputStream in = null;
+        FileContext fc = null;
+        try {
+          fc = FileContext.getFileContext(conf);
+          in = fc.open(fc.makeQualified(historyFilePath));
+        } catch (IOException ioe) {
+          LOG.info("Can not open history file: " + historyFilePath, ioe);
+          throw (new Exception("Can not open History File"));
+        }
 
-      JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
-          .getJobIndexInfo();
-      String jobhistoryFileName = FileNameIndexUtils
-          .getDoneFileName(jobIndexInfo);
-
-      Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
-      FSDataInputStream in = null;
-      FileContext fc = null;
-      try {
-        fc = FileContext.getFileContext(conf);
-        in = fc.open(fc.makeQualified(historyFilePath));
-      } catch (IOException ioe) {
-        LOG.info("Can not open history file: " + historyFilePath, ioe);
-        throw (new Exception("Can not open History File"));
+        parser = new JobHistoryParser(in);
+        jobInfo = parser.parse();
       }
-
-      JobHistoryParser parser = new JobHistoryParser(in);
-      JobInfo jobInfo = parser.parse();
       Exception parseException = parser.getParseException();
       Assert.assertNull("Caught an expected exception " + parseException,
           parseException);
@@ -464,29 +465,28 @@ public class TestJobHistoryParsing {
       // make sure all events are flushed
       app.waitForState(Service.STATE.STOPPED);
 
-      String jobhistoryDir = JobHistoryUtils
-          .getHistoryIntermediateDoneDirForUser(conf);
       JobHistory jobHistory = new JobHistory();
       jobHistory.init(conf);
 
-      JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
-          .getJobIndexInfo();
-      String jobhistoryFileName = FileNameIndexUtils
-          .getDoneFileName(jobIndexInfo);
+      HistoryFileInfo fileInfo = jobHistory.getJobFileInfo(jobId);
+      
+      JobHistoryParser parser;
+      JobInfo jobInfo;
+      synchronized (fileInfo) {
+        Path historyFilePath = fileInfo.getHistoryFile();
+        FSDataInputStream in = null;
+        FileContext fc = null;
+        try {
+          fc = FileContext.getFileContext(conf);
+          in = fc.open(fc.makeQualified(historyFilePath));
+        } catch (IOException ioe) {
+          LOG.info("Can not open history file: " + historyFilePath, ioe);
+          throw (new Exception("Can not open History File"));
+        }
 
-      Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
-      FSDataInputStream in = null;
-      FileContext fc = null;
-      try {
-        fc = FileContext.getFileContext(conf);
-        in = fc.open(fc.makeQualified(historyFilePath));
-      } catch (IOException ioe) {
-        LOG.info("Can not open history file: " + historyFilePath, ioe);
-        throw (new Exception("Can not open History File"));
+        parser = new JobHistoryParser(in);
+        jobInfo = parser.parse();
       }
-
-      JobHistoryParser parser = new JobHistoryParser(in);
-      JobInfo jobInfo = parser.parse();
       Exception parseException = parser.getParseException();
       Assert.assertNull("Caught an expected exception " + parseException,
           parseException);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java

@@ -304,7 +304,7 @@ public class TestClientRedirect {
     @Override
     public KillApplicationResponse forceKillApplication(
         KillApplicationRequest request) throws IOException {
-      return recordFactory.newRecordInstance(KillApplicationResponse.class);
+      return KillApplicationResponse.newInstance(true);
     }
 
     @Override

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