浏览代码

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-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
   BUG FIXES
 
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
     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-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
   OPTIMIZATIONS
 
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
     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-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
   OPTIMIZATIONS
 
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
     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
   BUG FIXES
 
 
@@ -470,6 +476,16 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10058. TestMetricsSystemImpl#testInitFirstVerifyStopInvokedImmediately
     HADOOP-10058. TestMetricsSystemImpl#testInitFirstVerifyStopInvokedImmediately
     fails on trunk (Chen He via jeagles)
     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
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

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

@@ -209,6 +209,10 @@
       <artifactId>protobuf-java</artifactId>
       <artifactId>protobuf-java</artifactId>
       <scope>compile</scope>
       <scope>compile</scope>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+    </dependency>
     <dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-auth</artifactId>
       <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>
    * </p>
    */
    */
   public static final RetryPolicy RETRY_FOREVER = new RetryForever();
   public static final RetryPolicy RETRY_FOREVER = new RetryForever();
-  
+
+  /**
+   * <p>
+   * Keep failing over forever
+   * </p>
+   */
+  public static final RetryPolicy FAILOVER_FOREVER = new FailoverForever();
+
   /**
   /**
    * <p>
    * <p>
    * Keep trying a limited number of times, waiting a fixed time between attempts,
    * Keep trying a limited number of times, waiting a fixed time between attempts,
@@ -166,6 +173,14 @@ public class RetryPolicies {
       return RetryAction.RETRY;
       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.
    * 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
       @Override
       public void run() {
       public void run() {
-        LOG.info("Starting " + getName());
+        LOG.info("Starting " + Thread.currentThread().getName());
         try {
         try {
           doRunLoop();
           doRunLoop();
         } finally {
         } finally {
           try {
           try {
             readSelector.close();
             readSelector.close();
           } catch (IOException ioe) {
           } 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) {
           } catch (InterruptedException e) {
             if (running) {                      // unexpected -- log it
             if (running) {                      // unexpected -- log it
-              LOG.info(getName() + " unexpectedly interrupted", e);
+              LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
             }
             }
           } catch (IOException ex) {
           } catch (IOException ex) {
             LOG.error("Error in Reader", ex);
             LOG.error("Error in Reader", ex);
@@ -620,7 +620,7 @@ public abstract class Server {
 
 
     @Override
     @Override
     public void run() {
     public void run() {
-      LOG.info(getName() + ": starting");
+      LOG.info(Thread.currentThread().getName() + ": starting");
       SERVER.set(Server.this);
       SERVER.set(Server.this);
       connectionManager.startIdleScan();
       connectionManager.startIdleScan();
       while (running) {
       while (running) {
@@ -652,7 +652,7 @@ public abstract class Server {
           closeCurrentConnection(key, e);
           closeCurrentConnection(key, e);
         }
         }
       }
       }
-      LOG.info("Stopping " + this.getName());
+      LOG.info("Stopping " + Thread.currentThread().getName());
 
 
       synchronized (this) {
       synchronized (this) {
         try {
         try {
@@ -710,14 +710,14 @@ public abstract class Server {
       try {
       try {
         count = c.readAndProcess();
         count = c.readAndProcess();
       } catch (InterruptedException ieo) {
       } catch (InterruptedException ieo) {
-        LOG.info(getName() + ": readAndProcess caught InterruptedException", ieo);
+        LOG.info(Thread.currentThread().getName() + ": readAndProcess caught InterruptedException", ieo);
         throw ieo;
         throw ieo;
       } catch (Exception e) {
       } catch (Exception e) {
         // a WrappedRpcServerException is an exception that has been sent
         // a WrappedRpcServerException is an exception that has been sent
         // to the client, so the stacktrace is unnecessary; any other
         // to the client, so the stacktrace is unnecessary; any other
         // exceptions are unexpected internal server errors and thus the
         // exceptions are unexpected internal server errors and thus the
         // stacktrace should be logged
         // stacktrace should be logged
-        LOG.info(getName() + ": readAndProcess from client " +
+        LOG.info(Thread.currentThread().getName() + ": readAndProcess from client " +
             c.getHostAddress() + " threw exception [" + e + "]",
             c.getHostAddress() + " threw exception [" + e + "]",
             (e instanceof WrappedRpcServerException) ? null : e);
             (e instanceof WrappedRpcServerException) ? null : e);
         count = -1; //so that the (count < 0) block is executed
         count = -1; //so that the (count < 0) block is executed
@@ -740,7 +740,7 @@ public abstract class Server {
         try {
         try {
           acceptChannel.socket().close();
           acceptChannel.socket().close();
         } catch (IOException e) {
         } 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) {
       for (Reader r : readers) {
@@ -773,16 +773,16 @@ public abstract class Server {
 
 
     @Override
     @Override
     public void run() {
     public void run() {
-      LOG.info(getName() + ": starting");
+      LOG.info(Thread.currentThread().getName() + ": starting");
       SERVER.set(Server.this);
       SERVER.set(Server.this);
       try {
       try {
         doRunLoop();
         doRunLoop();
       } finally {
       } finally {
-        LOG.info("Stopping " + this.getName());
+        LOG.info("Stopping " + Thread.currentThread().getName());
         try {
         try {
           writeSelector.close();
           writeSelector.close();
         } catch (IOException ioe) {
         } 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);
                   doAsyncWrite(key);
               }
               }
             } catch (IOException e) {
             } catch (IOException e) {
-              LOG.info(getName() + ": doAsyncWrite threw exception " + e);
+              LOG.info(Thread.currentThread().getName() + ": doAsyncWrite threw exception " + e);
             }
             }
           }
           }
           long now = Time.now();
           long now = Time.now();
@@ -918,7 +918,7 @@ public abstract class Server {
           call = responseQueue.removeFirst();
           call = responseQueue.removeFirst();
           SocketChannel channel = call.connection.channel;
           SocketChannel channel = call.connection.channel;
           if (LOG.isDebugEnabled()) {
           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
           // 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.
               done = false;            // more calls pending to be sent.
             }
             }
             if (LOG.isDebugEnabled()) {
             if (LOG.isDebugEnabled()) {
-              LOG.debug(getName() + ": responding to " + call
+              LOG.debug(Thread.currentThread().getName() + ": responding to " + call
                   + " Wrote " + numBytes + " bytes.");
                   + " Wrote " + numBytes + " bytes.");
             }
             }
           } else {
           } else {
@@ -965,7 +965,7 @@ public abstract class Server {
               }
               }
             }
             }
             if (LOG.isDebugEnabled()) {
             if (LOG.isDebugEnabled()) {
-              LOG.debug(getName() + ": responding to " + call
+              LOG.debug(Thread.currentThread().getName() + ": responding to " + call
                   + " Wrote partial " + numBytes + " bytes.");
                   + " Wrote partial " + numBytes + " bytes.");
             }
             }
           }
           }
@@ -973,7 +973,7 @@ public abstract class Server {
         }
         }
       } finally {
       } finally {
         if (error && call != null) {
         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.
           done = true;               // error. no more data for this channel.
           closeConnection(call.connection);
           closeConnection(call.connection);
         }
         }
@@ -2011,7 +2011,7 @@ public abstract class Server {
 
 
     @Override
     @Override
     public void run() {
     public void run() {
-      LOG.debug(getName() + ": starting");
+      LOG.debug(Thread.currentThread().getName() + ": starting");
       SERVER.set(Server.this);
       SERVER.set(Server.this);
       ByteArrayOutputStream buf = 
       ByteArrayOutputStream buf = 
         new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE);
         new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE);
@@ -2019,7 +2019,7 @@ public abstract class Server {
         try {
         try {
           final Call call = callQueue.take(); // pop the queue; maybe blocked here
           final Call call = callQueue.take(); // pop the queue; maybe blocked here
           if (LOG.isDebugEnabled()) {
           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 errorClass = null;
           String error = null;
           String error = null;
@@ -2052,7 +2052,7 @@ public abstract class Server {
             if (e instanceof UndeclaredThrowableException) {
             if (e instanceof UndeclaredThrowableException) {
               e = e.getCause();
               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) {
             if (e instanceof RuntimeException || e instanceof Error) {
               // These exception types indicate something is probably wrong
               // These exception types indicate something is probably wrong
               // on the server side, as opposed to just a normal exceptional
               // on the server side, as opposed to just a normal exceptional
@@ -2101,13 +2101,13 @@ public abstract class Server {
           }
           }
         } catch (InterruptedException e) {
         } catch (InterruptedException e) {
           if (running) {                          // unexpected -- log it
           if (running) {                          // unexpected -- log it
-            LOG.info(getName() + " unexpectedly interrupted", e);
+            LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
           }
           }
         } catch (Exception 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) {
   public void addSecretKey(Text alias, byte[] key) {
     secretKeysMap.put(alias, 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
    * Convenience method for reading a token storage file, and loading the Tokens
    * therein in the passed UGI
    * 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.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.security.Security;
 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.Map;
 import java.util.TreeMap;
 import java.util.TreeMap;
 
 
@@ -38,6 +42,7 @@ import javax.security.sasl.RealmCallback;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 import javax.security.sasl.SaslException;
 import javax.security.sasl.SaslServer;
 import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
 
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -63,6 +68,7 @@ public class SaslRpcServer {
   public static final String SASL_DEFAULT_REALM = "default";
   public static final String SASL_DEFAULT_REALM = "default";
   public static final Map<String, String> SASL_PROPS = 
   public static final Map<String, String> SASL_PROPS = 
       new TreeMap<String, String>();
       new TreeMap<String, String>();
+  private static SaslServerFactory saslFactory;
 
 
   public static enum QualityOfProtection {
   public static enum QualityOfProtection {
     AUTHENTICATION("auth"),
     AUTHENTICATION("auth"),
@@ -151,7 +157,7 @@ public class SaslRpcServer {
         new PrivilegedExceptionAction<SaslServer>() {
         new PrivilegedExceptionAction<SaslServer>() {
           @Override
           @Override
           public SaslServer run() throws SaslException  {
           public SaslServer run() throws SaslException  {
-            return Sasl.createSaslServer(mechanism, protocol, serverId,
+            return saslFactory.createSaslServer(mechanism, protocol, serverId,
                 SaslRpcServer.SASL_PROPS, callback);
                 SaslRpcServer.SASL_PROPS, callback);
           }
           }
         });
         });
@@ -180,6 +186,7 @@ public class SaslRpcServer {
     SASL_PROPS.put(Sasl.QOP, saslQOP.getSaslQop());
     SASL_PROPS.put(Sasl.QOP, saslQOP.getSaslQop());
     SASL_PROPS.put(Sasl.SERVER_AUTH, "true");
     SASL_PROPS.put(Sasl.SERVER_AUTH, "true");
     Security.addProvider(new SaslPlainServer.SecurityProvider());
     Security.addProvider(new SaslPlainServer.SecurityProvider());
+    saslFactory = new FastSaslServerFactory(SASL_PROPS);
   }
   }
   
   
   static String encodeIdentifier(byte[] identifier) {
   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 = 
     private static final AppConfigurationEntry[] SIMPLE_CONF = 
       new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN, HADOOP_LOGIN};
       new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN, HADOOP_LOGIN};
-
+    
     private static final AppConfigurationEntry[] USER_KERBEROS_CONF =
     private static final AppConfigurationEntry[] USER_KERBEROS_CONF =
       new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN, USER_KERBEROS_LOGIN,
       new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN, USER_KERBEROS_LOGIN,
                                   HADOOP_LOGIN};
                                   HADOOP_LOGIN};
@@ -682,44 +682,59 @@ public class UserGroupInformation {
   public synchronized 
   public synchronized 
   static UserGroupInformation getLoginUser() throws IOException {
   static UserGroupInformation getLoginUser() throws IOException {
     if (loginUser == null) {
     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
   @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
    * 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
    * @throws IOException
    */
    */
   @SuppressWarnings("unchecked")
   @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);
       proxy.sleep(pingInterval*4);
     } finally {
     } finally {
       if (proxy != null) RPC.stopProxy(proxy);
       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("Testing QOP:"+expectedQop);
     LOG.info("---------------------------------");
     LOG.info("---------------------------------");
     conf = new Configuration();
     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());
     conf.set("hadoop.rpc.protection", expectedQop.name().toLowerCase());
     UserGroupInformation.setConfiguration(conf);
     UserGroupInformation.setConfiguration(conf);
     enableSecretManager = null;
     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;
   private volatile boolean activeState;
   // The stream write-back status. True means one thread is doing write back.
   // The stream write-back status. True means one thread is doing write back.
   private volatile boolean asyncStatus;
   private volatile boolean asyncStatus;
+  private volatile long asyncWriteBackStartOffset;
 
 
   /**
   /**
    * The current offset of the file in HDFS. All the content before this offset
    * The current offset of the file in HDFS. All the content before this offset
@@ -209,6 +210,7 @@ class OpenFileCtx {
     updateLastAccessTime();
     updateLastAccessTime();
     activeState = true;
     activeState = true;
     asyncStatus = false;
     asyncStatus = false;
+    asyncWriteBackStartOffset = 0;
     dumpOut = null;
     dumpOut = null;
     raf = null;
     raf = null;
     nonSequentialWriteInMemory = new AtomicLong(0);
     nonSequentialWriteInMemory = new AtomicLong(0);
@@ -580,6 +582,7 @@ class OpenFileCtx {
               + nextOffset.get());
               + nextOffset.get());
         }
         }
         asyncStatus = true;
         asyncStatus = true;
+        asyncWriteBackStartOffset = writeCtx.getOffset();
         asyncDataService.execute(new AsyncDataService.WriteBackTask(this));
         asyncDataService.execute(new AsyncDataService.WriteBackTask(this));
       } else {
       } else {
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
@@ -903,9 +906,11 @@ class OpenFileCtx {
   /** Invoked by AsynDataService to write back to HDFS */
   /** Invoked by AsynDataService to write back to HDFS */
   void executeWriteBack() {
   void executeWriteBack() {
     Preconditions.checkState(asyncStatus,
     Preconditions.checkState(asyncStatus,
-        "The openFileCtx has false async status");
+        "openFileCtx has false asyncStatus, fileId:" + latestAttr.getFileid());
+    final long startOffset = asyncWriteBackStartOffset;  
     try {
     try {
       while (activeState) {
       while (activeState) {
+        // asyncStatus could be changed to false in offerNextToWrite()
         WriteCtx toWrite = offerNextToWrite();
         WriteCtx toWrite = offerNextToWrite();
         if (toWrite != null) {
         if (toWrite != null) {
           // Do the write
           // Do the write
@@ -921,8 +926,18 @@ class OpenFileCtx {
             + latestAttr.getFileId());
             + latestAttr.getFileId());
       }
       }
     } finally {
     } 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",
     return String.format("activeState: %b asyncStatus: %b nextOffset: %d",
         activeState, asyncStatus, nextOffset.get());
         activeState, asyncStatus, nextOffset.get());
   }
   }
-}
+}

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

@@ -216,36 +216,21 @@ Trunk (Unreleased)
     and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
     and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
     (jing9 via szetszwo)
     (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
     HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
     with DirectoryWithQuotaFeature.  (szetszwo)
     with DirectoryWithQuotaFeature.  (szetszwo)
 
 
     HDFS-5556. Add some more NameNode cache statistics, cache pool stats
     HDFS-5556. Add some more NameNode cache statistics, cache pool stats
     (cmccabe)
     (cmccabe)
 
 
-    HDFS-5545. Allow specifying endpoints for listeners in HttpServer. (Haohui
-    Mai via jing9)
-
     HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
     HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
 
 
     HDFS-5430. Support TTL on CacheDirectives. (wang)
     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-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
     HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
     FileWithSnapshotFeature.  (jing9 via szetszwo)
     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
     HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
     via jing9)
     via jing9)
 
 
@@ -253,6 +238,9 @@ Trunk (Unreleased)
     INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
     INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
     (jing9 via szetszwo)
     (jing9 via szetszwo)
 
 
+    HDFS-5431. Support cachepool-based limit management in path-based caching
+    (awang via cmccabe)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
     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
     HDFS-5454. DataNode UUID should be assigned prior to FsDataset
     initialization. (Arpit Agarwal)
     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
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
 
 
     HDFS-4985. Add storage type to the protocol and expose it in block report
     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.
     HDFS-5652. Refactor invalid block token exception handling in DFSInputStream.
     (Liang Xie via junping_du)
     (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
   OPTIMIZATIONS
 
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
     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.
     HDFS-5580. Fix infinite loop in Balancer.waitForMoveCompletion.
     (Binglin Chang via junping_du)
     (Binglin Chang via junping_du)
 
 
+    HDFS-5676. fix inconsistent synchronization of CachingStrategy (cmccabe)
+
 Release 2.3.0 - UNRELEASED
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -826,6 +850,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-5592. statechangeLog of completeFile should be logged only in case of success. 
     HDFS-5592. statechangeLog of completeFile should be logged only in case of success. 
     (Vinayakumar via umamahesh)
     (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
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -970,6 +997,11 @@ Release 2.3.0 - UNRELEASED
     HDFS-4201. NPE in BPServiceActor#sendHeartBeat. (jxiang via cmccabe)
     HDFS-4201. NPE in BPServiceActor#sendHeartBeat. (jxiang via cmccabe)
 
 
     HDFS-5666. Fix inconsistent synchronization in BPOfferService (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
 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" />
       <Method name="insertInternal" />
       <Bug pattern="BC_UNCONFIRMED_CAST" />
       <Bug pattern="BC_UNCONFIRMED_CAST" />
     </Match>
     </Match>
-    <!-- These two are used for shutting down and kicking the CRMon, do not need strong sync -->
     <Match>
     <Match>
       <Class name="org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor" />
       <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>
     </Match>
 
 
  </FindBugsFilter>
  </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;
 package org.apache.hadoop.hdfs;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.EnumSet;
 
 
 import org.apache.hadoop.fs.ByteBufferReadable;
 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.ClientMmap;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -89,10 +91,10 @@ public interface BlockReader extends ByteBufferReadable {
   /**
   /**
    * Get a ClientMmap object for this BlockReader.
    * 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
    * @return              The ClientMmap object, or null if mmap is not
    *                      supported.
    *                      supported.
    */
    */
-  ClientMmap getClientMmap(LocatedBlock curBlock,
+  ClientMmap getClientMmap(EnumSet<ReadOption> opts,
         ClientMmapManager mmapManager);
         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.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
@@ -98,7 +99,7 @@ public class BlockReaderFactory {
         // enabled, try to set up a BlockReaderLocal.
         // enabled, try to set up a BlockReaderLocal.
         BlockReader reader = newShortCircuitBlockReader(conf, file,
         BlockReader reader = newShortCircuitBlockReader(conf, file,
             block, blockToken, startOffset, len, peer, datanodeID,
             block, blockToken, startOffset, len, peer, datanodeID,
-            domSockFactory, verifyChecksum, fisCache);
+            domSockFactory, verifyChecksum, fisCache, cachingStrategy);
         if (reader != null) {
         if (reader != null) {
           // One we've constructed the short-circuit block reader, we don't
           // 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.
           // 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.
    * @param verifyChecksum     True if we should verify the checksums.
    *                           Note: even if this is true, when
    *                           Note: even if this is true, when
    *                           DFS_CLIENT_READ_CHECKSUM_SKIP_CHECKSUM_KEY is
    *                           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
    * @return                   The BlockReaderLocal, or null if the
    *                           DataNode declined to provide short-circuit
    *                           DataNode declined to provide short-circuit
@@ -172,7 +174,8 @@ public class BlockReaderFactory {
       Token<BlockTokenIdentifier> blockToken, long startOffset,
       Token<BlockTokenIdentifier> blockToken, long startOffset,
       long len, Peer peer, DatanodeID datanodeID,
       long len, Peer peer, DatanodeID datanodeID,
       DomainSocketFactory domSockFactory, boolean verifyChecksum,
       DomainSocketFactory domSockFactory, boolean verifyChecksum,
-      FileInputStreamCache fisCache) throws IOException {
+      FileInputStreamCache fisCache,
+      CachingStrategy cachingStrategy) throws IOException {
     final DataOutputStream out =
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(
         new DataOutputStream(new BufferedOutputStream(
           peer.getOutputStream()));
           peer.getOutputStream()));
@@ -189,9 +192,18 @@ public class BlockReaderFactory {
       FileInputStream fis[] = new FileInputStream[2];
       FileInputStream fis[] = new FileInputStream[2];
       sock.recvFileInputStreams(fis, buf, 0, buf.length);
       sock.recvFileInputStreams(fis, buf, 0, buf.length);
       try {
       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 {
       } finally {
         if (reader == null) {
         if (reader == null) {
           IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
           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.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collections;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Map;
 
 
+import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.client.ClientMmap;
 import org.apache.hadoop.hdfs.client.ClientMmap;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -706,8 +708,8 @@ class BlockReaderLocalLegacy implements BlockReader {
   }
   }
 
 
   @Override
   @Override
-  public ClientMmap getClientMmap(LocatedBlock curBlock,
-      ClientMmapManager mmapManager) {
+  public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
+        ClientMmapManager mmapManager) {
     return null;
     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.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStorageLocation;
 import org.apache.hadoop.fs.BlockStorageLocation;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 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.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.AclEntry;
 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.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 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.CacheDirectiveIterator;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 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.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -284,6 +285,8 @@ public class DFSClient implements java.io.Closeable {
     final boolean getHdfsBlocksMetadataEnabled;
     final boolean getHdfsBlocksMetadataEnabled;
     final int getFileBlockStorageLocationsNumThreads;
     final int getFileBlockStorageLocationsNumThreads;
     final int getFileBlockStorageLocationsTimeout;
     final int getFileBlockStorageLocationsTimeout;
+    final int retryTimesForGetLastBlockLength;
+    final int retryIntervalForGetLastBlockLength;
 
 
     final boolean useLegacyBlockReader;
     final boolean useLegacyBlockReader;
     final boolean useLegacyBlockReaderLocal;
     final boolean useLegacyBlockReaderLocal;
@@ -357,6 +360,12 @@ public class DFSClient implements java.io.Closeable {
       getFileBlockStorageLocationsTimeout = conf.getInt(
       getFileBlockStorageLocationsTimeout = conf.getInt(
           DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT,
           DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT,
           DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_DEFAULT);
           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(
       useLegacyBlockReader = conf.getBoolean(
           DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER,
           DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER,
@@ -2297,20 +2306,20 @@ public class DFSClient implements java.io.Closeable {
   }
   }
 
 
   public long addCacheDirective(
   public long addCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
     checkOpen();
     try {
     try {
-      return namenode.addCacheDirective(info);
+      return namenode.addCacheDirective(info, flags);
     } catch (RemoteException re) {
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
       throw re.unwrapRemoteException();
     }
     }
   }
   }
   
   
   public void modifyCacheDirective(
   public void modifyCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
     checkOpen();
     try {
     try {
-      namenode.modifyCacheDirective(info);
+      namenode.modifyCacheDirective(info, flags);
     } catch (RemoteException re) {
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
       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 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 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 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
   // HA related configuration
   public static final String  DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX = "dfs.client.failover.proxy.provider";
   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.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.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.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.io.ByteBufferPool;
 import org.apache.hadoop.io.ByteBufferPool;
@@ -227,7 +228,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         dfsClient.getConf().shortCircuitStreamsCacheSize,
         dfsClient.getConf().shortCircuitStreamsCacheSize,
         dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
         dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
     this.cachingStrategy =
     this.cachingStrategy =
-        dfsClient.getDefaultReadCachingStrategy().duplicate();
+        dfsClient.getDefaultReadCachingStrategy();
     openInfo();
     openInfo();
   }
   }
 
 
@@ -236,7 +237,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    */
    */
   synchronized void openInfo() throws IOException, UnresolvedLinkException {
   synchronized void openInfo() throws IOException, UnresolvedLinkException {
     lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
     lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
-    int retriesForLastBlockLength = 3;
+    int retriesForLastBlockLength = dfsClient.getConf().retryTimesForGetLastBlockLength;
     while (retriesForLastBlockLength > 0) {
     while (retriesForLastBlockLength > 0) {
       // Getting last block length as -1 is a special case. When cluster
       // Getting last block length as -1 is a special case. When cluster
       // restarts, DNs may not report immediately. At this time partial block
       // 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. "
         DFSClient.LOG.warn("Last block locations not available. "
             + "Datanodes might not have reported blocks completely."
             + "Datanodes might not have reported blocks completely."
             + " Will retry for " + retriesForLastBlockLength + " times");
             + " Will retry for " + retriesForLastBlockLength + " times");
-        waitFor(4000);
+        waitFor(dfsClient.getConf().retryIntervalForGetLastBlockLength);
         lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
         lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
       } else {
       } else {
         break;
         break;
@@ -573,7 +574,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
         Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
         blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
         blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
             accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
             accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
-            buffersize, verifyChecksum, dfsClient.clientName);
+            buffersize, verifyChecksum, dfsClient.clientName, cachingStrategy);
         if(connectFailedOnce) {
         if(connectFailedOnce) {
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
                              " for " + blk);
                              " for " + blk);
@@ -927,7 +928,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       // cached block locations may have been updated by chooseDataNode()
       // cached block locations may have been updated by chooseDataNode()
       // or fetchBlockAt(). Always get the latest list of locations at the 
       // or fetchBlockAt(). Always get the latest list of locations at the 
       // start of the loop.
       // start of the loop.
-      block = getBlockAt(block.getStartOffset(), false);
+      CachingStrategy curCachingStrategy;
+      synchronized (this) {
+        block = getBlockAt(block.getStartOffset(), false);
+        curCachingStrategy = cachingStrategy;
+      }
       DNAddrPair retval = chooseDataNode(block);
       DNAddrPair retval = chooseDataNode(block);
       DatanodeInfo chosenNode = retval.info;
       DatanodeInfo chosenNode = retval.info;
       InetSocketAddress targetAddr = retval.addr;
       InetSocketAddress targetAddr = retval.addr;
@@ -939,7 +944,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         int len = (int) (end - start + 1);
         int len = (int) (end - start + 1);
         reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
         reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
             blockToken, start, len, buffersize, verifyChecksum,
             blockToken, start, len, buffersize, verifyChecksum,
-            dfsClient.clientName);
+            dfsClient.clientName, curCachingStrategy);
         int nread = reader.readAll(buf, offset, len);
         int nread = reader.readAll(buf, offset, len);
         if (nread != len) {
         if (nread != len) {
           throw new IOException("truncated return from reader.read(): " +
           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 bufferSize  The IO buffer size (not the client buffer size)
    * @param verifyChecksum  Whether to verify checksum
    * @param verifyChecksum  Whether to verify checksum
    * @param clientName  Client name
    * @param clientName  Client name
+   * @param CachingStrategy  caching strategy to use
    * @return New BlockReader instance
    * @return New BlockReader instance
    */
    */
   protected BlockReader getBlockReader(InetSocketAddress dnAddr,
   protected BlockReader getBlockReader(InetSocketAddress dnAddr,
@@ -1063,7 +1069,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
                                        long len,
                                        long len,
                                        int bufferSize,
                                        int bufferSize,
                                        boolean verifyChecksum,
                                        boolean verifyChecksum,
-                                       String clientName)
+                                       String clientName,
+                                       CachingStrategy curCachingStrategy)
       throws IOException {
       throws IOException {
     // Firstly, we check to see if we have cached any file descriptors for
     // 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.
     // 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 " +
         DFSClient.LOG.debug("got FileInputStreams for " + block + " from " +
             "the FileInputStreamCache.");
             "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
     // 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,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             len, verifyChecksum, clientName, peer, chosenNode, 
             len, verifyChecksum, clientName, peer, chosenNode, 
             dsFactory, peerCache, fileInputStreamCache,
             dsFactory, peerCache, fileInputStreamCache,
-            allowShortCircuitLocalReads, cachingStrategy);
+            allowShortCircuitLocalReads, curCachingStrategy);
         return reader;
         return reader;
       } catch (IOException ex) {
       } catch (IOException ex) {
         DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
         DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
@@ -1132,7 +1148,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             len, verifyChecksum, clientName, peer, chosenNode,
             len, verifyChecksum, clientName, peer, chosenNode,
             dsFactory, peerCache, fileInputStreamCache,
             dsFactory, peerCache, fileInputStreamCache,
-            allowShortCircuitLocalReads, cachingStrategy);
+            allowShortCircuitLocalReads, curCachingStrategy);
         return reader;
         return reader;
       } catch (IOException e) {
       } catch (IOException e) {
         DFSClient.LOG.warn("failed to connect to " + domSock, e);
         DFSClient.LOG.warn("failed to connect to " + domSock, e);
@@ -1156,7 +1172,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             len, verifyChecksum, clientName, peer, chosenNode, 
             len, verifyChecksum, clientName, peer, chosenNode, 
             dsFactory, peerCache, fileInputStreamCache, false,
             dsFactory, peerCache, fileInputStreamCache, false,
-            cachingStrategy);
+            curCachingStrategy);
         return reader;
         return reader;
       } catch (IOException ex) {
       } catch (IOException ex) {
         DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
         DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
@@ -1176,7 +1192,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         dfsClient.getConf(), file, block, blockToken, startOffset,
         dfsClient.getConf(), file, block, blockToken, startOffset,
         len, verifyChecksum, clientName, peer, chosenNode, 
         len, verifyChecksum, clientName, peer, chosenNode, 
         dsFactory, peerCache, fileInputStreamCache, false,
         dsFactory, peerCache, fileInputStreamCache, false,
-        cachingStrategy);
+        curCachingStrategy);
   }
   }
 
 
 
 
@@ -1450,14 +1466,18 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   @Override
   @Override
   public synchronized void setReadahead(Long readahead)
   public synchronized void setReadahead(Long readahead)
       throws IOException {
       throws IOException {
-    this.cachingStrategy.setReadahead(readahead);
+    this.cachingStrategy =
+        new CachingStrategy.Builder(this.cachingStrategy).
+            setReadahead(readahead).build();
     closeCurrentBlockReader();
     closeCurrentBlockReader();
   }
   }
 
 
   @Override
   @Override
   public synchronized void setDropBehind(Boolean dropBehind)
   public synchronized void setDropBehind(Boolean dropBehind)
       throws IOException {
       throws IOException {
-    this.cachingStrategy.setDropBehind(dropBehind);
+    this.cachingStrategy =
+        new CachingStrategy.Builder(this.cachingStrategy).
+            setDropBehind(dropBehind).build();
     closeCurrentBlockReader();
     closeCurrentBlockReader();
   }
   }
 
 
@@ -1479,23 +1499,19 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             "at position " + pos);
             "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) {
     if (buffer != null) {
       extendedReadBuffers.put(buffer, bufferPool);
       extendedReadBuffers.put(buffer, bufferPool);
     }
     }
     return buffer;
     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
     // Java ByteBuffers can't be longer than 2 GB, because they use
     // 4-byte signed integers to represent capacity, etc.
     // 4-byte signed integers to represent capacity, etc.
     // So we can't mmap the parts of the block higher than the 2 GB offset.
     // 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 blockPos = curPos - blockStartInFile;
     long limit = blockPos + length;
     long limit = blockPos + length;
     ClientMmap clientMmap =
     ClientMmap clientMmap =
-        blockReader.getClientMmap(currentLocatedBlock,
-            dfsClient.getMmapManager());
+        blockReader.getClientMmap(opts, dfsClient.getMmapManager());
     if (clientMmap == null) {
     if (clientMmap == null) {
       if (DFSClient.LOG.isDebugEnabled()) {
       if (DFSClient.LOG.isDebugEnabled()) {
         DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
         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 Progressable progress;
   private final short blockReplication; // replication factor of file
   private final short blockReplication; // replication factor of file
   private boolean shouldSyncBlock = false; // force blocks to disk upon close
   private boolean shouldSyncBlock = false; // force blocks to disk upon close
-  private CachingStrategy cachingStrategy;
+  private AtomicReference<CachingStrategy> cachingStrategy;
   private boolean failPacket = false;
   private boolean failPacket = false;
   
   
   private static class Packet {
   private static class Packet {
@@ -1183,7 +1183,7 @@ public class DFSOutputStream extends FSOutputSummer
           new Sender(out).writeBlock(block, accessToken, dfsClient.clientName,
           new Sender(out).writeBlock(block, accessToken, dfsClient.clientName,
               nodes, null, recoveryFlag? stage.getRecoveryStage() : stage, 
               nodes, null, recoveryFlag? stage.getRecoveryStage() : stage, 
               nodes.length, block.getNumBytes(), bytesSent, newGS, checksum,
               nodes.length, block.getNumBytes(), bytesSent, newGS, checksum,
-              cachingStrategy);
+              cachingStrategy.get());
   
   
           // receive ack for connect
           // receive ack for connect
           BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
           BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
@@ -1378,8 +1378,8 @@ public class DFSOutputStream extends FSOutputSummer
     this.blockSize = stat.getBlockSize();
     this.blockSize = stat.getBlockSize();
     this.blockReplication = stat.getReplication();
     this.blockReplication = stat.getReplication();
     this.progress = progress;
     this.progress = progress;
-    this.cachingStrategy =
-        dfsClient.getDefaultWriteCachingStrategy().duplicate();
+    this.cachingStrategy = new AtomicReference<CachingStrategy>(
+        dfsClient.getDefaultWriteCachingStrategy());
     if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
     if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug(
       DFSClient.LOG.debug(
           "Set non-null progress callback on DFSOutputStream " + src);
           "Set non-null progress callback on DFSOutputStream " + src);
@@ -1993,7 +1993,14 @@ public class DFSOutputStream extends FSOutputSummer
 
 
   @Override
   @Override
   public void setDropBehind(Boolean dropBehind) throws IOException {
   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
   @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.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStorageLocation;
 import org.apache.hadoop.fs.BlockStorageLocation;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -1587,40 +1588,56 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
     }.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.
    * Add a new CacheDirective.
    * 
    * 
    * @param info Information about a directive to add.
    * @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.
    * @return the ID of the directive that was created.
    * @throws IOException if the directive could not be added
    * @throws IOException if the directive could not be added
    */
    */
   public long addCacheDirective(
   public long addCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     Preconditions.checkNotNull(info.getPath());
     Preconditions.checkNotNull(info.getPath());
     Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
     Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
         makeQualified(getUri(), getWorkingDirectory());
         makeQualified(getUri(), getWorkingDirectory());
     return dfs.addCacheDirective(
     return dfs.addCacheDirective(
         new CacheDirectiveInfo.Builder(info).
         new CacheDirectiveInfo.Builder(info).
             setPath(path).
             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.
    * 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
    * @throws IOException if the directive could not be modified
    */
    */
   public void modifyCacheDirective(
   public void modifyCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     if (info.getPath() != null) {
     if (info.getPath() != null) {
       info = new CacheDirectiveInfo.Builder(info).
       info = new CacheDirectiveInfo.Builder(info).
           setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
           setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
               makeQualified(getUri(), getWorkingDirectory())).build();
               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.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FSInputChecker;
 import org.apache.hadoop.fs.FSInputChecker;
 import org.apache.hadoop.fs.Path;
 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.ClientMmap;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.Peer;
@@ -490,8 +492,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   }
   }
 
 
   @Override
   @Override
-  public ClientMmap getClientMmap(LocatedBlock curBlock,
-      ClientMmapManager mmapManager) {
+  public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
+        ClientMmapManager mmapManager) {
     return null;
     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.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.nio.channels.ReadableByteChannel;
 import java.nio.channels.ReadableByteChannel;
+import java.util.EnumSet;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 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.ClientMmap;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.Peer;
@@ -455,8 +457,8 @@ public class RemoteBlockReader2  implements BlockReader {
   }
   }
 
 
   @Override
   @Override
-  public ClientMmap getClientMmap(LocatedBlock curBlock,
-      ClientMmapManager manager) {
+  public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
+        ClientMmapManager mmapManager) {
     return null;
     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.io.IOException;
 import java.net.URI;
 import java.net.URI;
+import java.util.EnumSet;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
@@ -131,25 +133,26 @@ public class HdfsAdmin {
    * Add a new CacheDirectiveInfo.
    * Add a new CacheDirectiveInfo.
    * 
    * 
    * @param info Information about a directive to add.
    * @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.
    * @return the ID of the directive that was created.
    * @throws IOException if the directive could not be added
    * @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.
    * 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
    * @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;
 package org.apache.hadoop.hdfs.protocol;
 
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import javax.annotation.Nullable;
 import javax.annotation.Nullable;
@@ -32,14 +30,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.permission.FsPermission;
 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.
  * CachePoolInfo describes a cache pool.
@@ -64,7 +54,7 @@ public class CachePoolInfo {
   FsPermission mode;
   FsPermission mode;
 
 
   @Nullable
   @Nullable
-  Integer weight;
+  Long limit;
 
 
   public CachePoolInfo(String poolName) {
   public CachePoolInfo(String poolName) {
     this.poolName = poolName;
     this.poolName = poolName;
@@ -101,12 +91,12 @@ public class CachePoolInfo {
     return this;
     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;
     return this;
   }
   }
 
 
@@ -117,7 +107,7 @@ public class CachePoolInfo {
       append(", groupName:").append(groupName).
       append(", groupName:").append(groupName).
       append(", mode:").append((mode == null) ? "null" :
       append(", mode:").append((mode == null) ? "null" :
           String.format("0%03o", mode.toShort())).
           String.format("0%03o", mode.toShort())).
-      append(", weight:").append(weight).
+      append(", limit:").append(limit).
       append("}").toString();
       append("}").toString();
   }
   }
   
   
@@ -134,7 +124,7 @@ public class CachePoolInfo {
         append(ownerName, other.ownerName).
         append(ownerName, other.ownerName).
         append(groupName, other.groupName).
         append(groupName, other.groupName).
         append(mode, other.mode).
         append(mode, other.mode).
-        append(weight, other.weight).
+        append(limit, other.limit).
         isEquals();
         isEquals();
   }
   }
 
 
@@ -145,7 +135,7 @@ public class CachePoolInfo {
         append(ownerName).
         append(ownerName).
         append(groupName).
         append(groupName).
         append(mode).
         append(mode).
-        append(weight).
+        append(limit).
         hashCode();
         hashCode();
   }
   }
 
 
@@ -153,8 +143,8 @@ public class CachePoolInfo {
     if (info == null) {
     if (info == null) {
       throw new InvalidRequestException("CachePoolInfo is 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);
     validateName(info.poolName);
   }
   }
@@ -167,66 +157,4 @@ public class CachePoolInfo {
       throw new IOException("invalid empty cache pool name");
       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 {
   public static class Builder {
     private long bytesNeeded;
     private long bytesNeeded;
     private long bytesCached;
     private long bytesCached;
+    private long bytesOverlimit;
     private long filesNeeded;
     private long filesNeeded;
     private long filesCached;
     private long filesCached;
 
 
@@ -46,6 +47,11 @@ public class CachePoolStats {
       return this;
       return this;
     }
     }
 
 
+    public Builder setBytesOverlimit(long bytesOverlimit) {
+      this.bytesOverlimit = bytesOverlimit;
+      return this;
+    }
+
     public Builder setFilesNeeded(long filesNeeded) {
     public Builder setFilesNeeded(long filesNeeded) {
       this.filesNeeded = filesNeeded;
       this.filesNeeded = filesNeeded;
       return this;
       return this;
@@ -57,20 +63,22 @@ public class CachePoolStats {
     }
     }
 
 
     public CachePoolStats build() {
     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 bytesNeeded;
   private final long bytesCached;
   private final long bytesCached;
+  private final long bytesOverlimit;
   private final long filesNeeded;
   private final long filesNeeded;
   private final long filesCached;
   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.bytesNeeded = bytesNeeded;
     this.bytesCached = bytesCached;
     this.bytesCached = bytesCached;
+    this.bytesOverlimit = bytesOverlimit;
     this.filesNeeded = filesNeeded;
     this.filesNeeded = filesNeeded;
     this.filesCached = filesCached;
     this.filesCached = filesCached;
   }
   }
@@ -83,6 +91,10 @@ public class CachePoolStats {
     return bytesCached;
     return bytesCached;
   }
   }
 
 
+  public long getBytesOverlimit() {
+    return bytesOverlimit;
+  }
+
   public long getFilesNeeded() {
   public long getFilesNeeded() {
     return filesNeeded;
     return filesNeeded;
   }
   }
@@ -95,6 +107,7 @@ public class CachePoolStats {
     return new StringBuilder().append("{").
     return new StringBuilder().append("{").
       append("bytesNeeded:").append(bytesNeeded).
       append("bytesNeeded:").append(bytesNeeded).
       append(", bytesCached:").append(bytesCached).
       append(", bytesCached:").append(bytesCached).
+      append(", bytesOverlimit:").append(bytesOverlimit).
       append(", filesNeeded:").append(filesNeeded).
       append(", filesNeeded:").append(filesNeeded).
       append(", filesCached:").append(filesCached).
       append(", filesCached:").append(filesCached).
       append("}").toString();
       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.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -1103,23 +1105,24 @@ public interface ClientProtocol {
    * Add a CacheDirective to the CacheManager.
    * Add a CacheDirective to the CacheManager.
    * 
    * 
    * @param directive A CacheDirectiveInfo to be added
    * @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
    * @return A CacheDirectiveInfo associated with the added directive
    * @throws IOException if the directive could not be added
    * @throws IOException if the directive could not be added
    */
    */
   @AtMostOnce
   @AtMostOnce
-  public long addCacheDirective(
-      CacheDirectiveInfo directive) throws IOException;
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
 
 
   /**
   /**
    * Modify a CacheDirective in the CacheManager.
    * 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
    * @throws IOException if the directive could not be modified
    */
    */
   @AtMostOnce
   @AtMostOnce
-  public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException;
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
 
 
   /**
   /**
    * Remove a CacheDirectiveInfo from the CacheManager.
    * 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 {
     try {
       HdfsFileStatus result = server.create(req.getSrc(),
       HdfsFileStatus result = server.create(req.getSrc(),
           PBHelper.convert(req.getMasked()), req.getClientName(),
           PBHelper.convert(req.getMasked()), req.getClientName(),
-          PBHelper.convert(req.getCreateFlag()), req.getCreateParent(),
+          PBHelper.convertCreateFlag(req.getCreateFlag()), req.getCreateParent(),
           (short) req.getReplication(), req.getBlockSize());
           (short) req.getReplication(), req.getBlockSize());
 
 
       if (result != null) {
       if (result != null) {
@@ -1064,9 +1064,11 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, AddCacheDirectiveRequestProto request)
       RpcController controller, AddCacheDirectiveRequestProto request)
       throws ServiceException {
       throws ServiceException {
     try {
     try {
+      long id = server.addCacheDirective(
+          PBHelper.convert(request.getInfo()),
+          PBHelper.convertCacheFlags(request.getCacheFlags()));
       return AddCacheDirectiveResponseProto.newBuilder().
       return AddCacheDirectiveResponseProto.newBuilder().
-              setId(server.addCacheDirective(
-                  PBHelper.convert(request.getInfo()))).build();
+              setId(id).build();
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);
     }
     }
@@ -1078,7 +1080,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
       throws ServiceException {
     try {
     try {
       server.modifyCacheDirective(
       server.modifyCacheDirective(
-          PBHelper.convert(request.getInfo()));
+          PBHelper.convert(request.getInfo()),
+          PBHelper.convertCacheFlags(request.getCacheFlags()));
       return ModifyCacheDirectiveResponseProto.newBuilder().build();
       return ModifyCacheDirectiveResponseProto.newBuilder().build();
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(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.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -1012,24 +1014,32 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public long addCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
     try {
     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) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
 
 
   @Override
   @Override
-  public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
     try {
     try {
-      rpcProxy.modifyCacheDirective(null,
+      ModifyCacheDirectiveRequestProto.Builder builder =
           ModifyCacheDirectiveRequestProto.newBuilder().
           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) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(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.EnumSet;
 import java.util.List;
 import java.util.List;
 
 
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 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.CacheDirectiveEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
 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.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.CachePoolEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
@@ -1205,7 +1207,7 @@ public class PBHelper {
     return value;
     return value;
   }
   }
   
   
-  public static EnumSetWritable<CreateFlag> convert(int flag) {
+  public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
     EnumSet<CreateFlag> result = 
     EnumSet<CreateFlag> result = 
        EnumSet.noneOf(CreateFlag.class);   
        EnumSet.noneOf(CreateFlag.class);   
     if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
     if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
@@ -1220,7 +1222,23 @@ public class PBHelper {
     }
     }
     return new EnumSetWritable<CreateFlag>(result);
     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) {
   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
     if (fs == null)
     if (fs == null)
       return null;
       return null;
@@ -1818,8 +1836,8 @@ public class PBHelper {
     if (info.getMode() != null) {
     if (info.getMode() != null) {
       builder.setMode(info.getMode().toShort());
       builder.setMode(info.getMode().toShort());
     }
     }
-    if (info.getWeight() != null) {
-      builder.setWeight(info.getWeight());
+    if (info.getLimit() != null) {
+      builder.setLimit(info.getLimit());
     }
     }
     return builder.build();
     return builder.build();
   }
   }
@@ -1837,8 +1855,8 @@ public class PBHelper {
     if (proto.hasMode()) {
     if (proto.hasMode()) {
       info.setMode(new FsPermission((short)proto.getMode()));
       info.setMode(new FsPermission((short)proto.getMode()));
     }
     }
-    if (proto.hasWeight()) {
-      info.setWeight(proto.getWeight());
+    if (proto.hasLimit())  {
+      info.setLimit(proto.getLimit());
     }
     }
     return info;
     return info;
   }
   }
@@ -1847,6 +1865,7 @@ public class PBHelper {
     CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
     CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
     builder.setBytesNeeded(stats.getBytesNeeded());
     builder.setBytesNeeded(stats.getBytesNeeded());
     builder.setBytesCached(stats.getBytesCached());
     builder.setBytesCached(stats.getBytesCached());
+    builder.setBytesOverlimit(stats.getBytesOverlimit());
     builder.setFilesNeeded(stats.getFilesNeeded());
     builder.setFilesNeeded(stats.getFilesNeeded());
     builder.setFilesCached(stats.getFilesCached());
     builder.setFilesCached(stats.getFilesCached());
     return builder.build();
     return builder.build();
@@ -1856,6 +1875,7 @@ public class PBHelper {
     CachePoolStats.Builder builder = new CachePoolStats.Builder();
     CachePoolStats.Builder builder = new CachePoolStats.Builder();
     builder.setBytesNeeded(proto.getBytesNeeded());
     builder.setBytesNeeded(proto.getBytesNeeded());
     builder.setBytesCached(proto.getBytesCached());
     builder.setBytesCached(proto.getBytesCached());
+    builder.setBytesOverlimit(proto.getBytesOverlimit());
     builder.setFilesNeeded(proto.getFilesNeeded());
     builder.setFilesNeeded(proto.getFilesNeeded());
     builder.setFilesCached(proto.getFilesCached());
     builder.setFilesCached(proto.getFilesCached());
     return builder.build();
     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 isReplicationInProgress(DatanodeDescriptor srcNode) {
     boolean status = false;
     boolean status = false;
+    boolean firstReplicationLog = true;
     int underReplicatedBlocks = 0;
     int underReplicatedBlocks = 0;
     int decommissionOnlyReplicas = 0;
     int decommissionOnlyReplicas = 0;
     int underReplicatedInOpenFiles = 0;
     int underReplicatedInOpenFiles = 0;
@@ -2907,10 +2908,17 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         int curExpectedReplicas = getReplication(block);
         int curExpectedReplicas = getReplication(block);
         if (isNeededReplication(block, curExpectedReplicas, curReplicas)) {
         if (isNeededReplication(block, curExpectedReplicas, curReplicas)) {
           if (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) {
             if (!status) {
               status = true;
               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++;
             underReplicatedBlocks++;
             if ((curReplicas == 0) && (num.decommissionedReplicas() > 0)) {
             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.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Random;
 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.Log;
 import org.apache.commons.logging.LogFactory;
 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.GSet;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
+import com.google.common.base.Preconditions;
+
 /**
 /**
  * Scans the namesystem, scheduling blocks to be cached as appropriate.
  * 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;
   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.
    * Cache directives found in the previous scan.
@@ -108,7 +140,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
    * Blocks found in the previous scan.
    * Blocks found in the previous scan.
    */
    */
   private long scannedBlocks;
   private long scannedBlocks;
-  
+
   public CacheReplicationMonitor(FSNamesystem namesystem,
   public CacheReplicationMonitor(FSNamesystem namesystem,
       CacheManager cacheManager, long intervalMs) {
       CacheManager cacheManager, long intervalMs) {
     this.namesystem = namesystem;
     this.namesystem = namesystem;
@@ -120,41 +152,60 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
 
 
   @Override
   @Override
   public void run() {
   public void run() {
-    shutdown = false;
-    rescanImmediately = true;
-    scanTimeMs = 0;
+    startTimeMs = 0;
     LOG.info("Starting CacheReplicationMonitor with interval " +
     LOG.info("Starting CacheReplicationMonitor with interval " +
              intervalMs + " milliseconds");
              intervalMs + " milliseconds");
     try {
     try {
       long curTimeMs = Time.monotonicNow();
       long curTimeMs = Time.monotonicNow();
       while (true) {
       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) {
           while (true) {
             if (shutdown) {
             if (shutdown) {
               LOG.info("Shutting down CacheReplicationMonitor");
               LOG.info("Shutting down CacheReplicationMonitor");
               return;
               return;
             }
             }
-            if (rescanImmediately) {
-              LOG.info("Rescanning on request");
-              rescanImmediately = false;
+            if (needsRescan) {
+              LOG.info("Rescanning because of pending operations");
               break;
               break;
             }
             }
-            long delta = (scanTimeMs + intervalMs) - curTimeMs;
+            long delta = (startTimeMs + intervalMs) - curTimeMs;
             if (delta <= 0) {
             if (delta <= 0) {
-              LOG.info("Rescanning after " + (curTimeMs - scanTimeMs) +
+              LOG.info("Rescanning after " + (curTimeMs - startTimeMs) +
                   " milliseconds");
                   " milliseconds");
               break;
               break;
             }
             }
-            this.wait(delta);
+            doRescan.await(delta, TimeUnit.MILLISECONDS);
             curTimeMs = Time.monotonicNow();
             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;
         mark = !mark;
         rescan();
         rescan();
         curTimeMs = Time.monotonicNow();
         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 " +
         LOG.info("Scanned " + scannedDirectives + " directive(s) and " +
-            scannedBlocks + " block(s) in " + (curTimeMs - scanTimeMs) + " " +
+            scannedBlocks + " block(s) in " + (curTimeMs - startTimeMs) + " " +
             "millisecond(s).");
             "millisecond(s).");
       }
       }
     } catch (Throwable t) {
     } 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
   @Override
   public void close() throws IOException {
   public void close() throws IOException {
-    synchronized(this) {
+    lock.lock();
+    try {
       if (shutdown) return;
       if (shutdown) return;
       shutdown = true;
       shutdown = true;
-      this.notifyAll();
+      doRescan.signalAll();
+      scanFinished.signalAll();
+    } finally {
+      lock.unlock();
     }
     }
     try {
     try {
       if (this.isAlive()) {
       if (this.isAlive()) {
@@ -228,12 +359,14 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       // Reset the directive's statistics
       // Reset the directive's statistics
       directive.resetStatistics();
       directive.resetStatistics();
       // Skip processing this entry if it has expired
       // 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 (directive.getExpiryTime() > 0 && directive.getExpiryTime() <= now) {
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Skipping directive id " + directive.getId()
           LOG.debug("Skipping directive id " + directive.getId()
               + " because it has expired (" + directive.getExpiryTime() + ">="
               + " because it has expired (" + directive.getExpiryTime() + ">="
-              + now);
+              + now + ")");
         }
         }
         continue;
         continue;
       }
       }
@@ -280,15 +413,27 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
 
 
     // Increment the "needed" statistics
     // Increment the "needed" statistics
     directive.addFilesNeeded(1);
     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);
     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;
     long cachedTotal = 0;
     for (BlockInfo blockInfo : blockInfos) {
     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;
       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.
    * 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 final class Url {
     public static String authority(String scheme, DatanodeID d) {
     public static String authority(String scheme, DatanodeID d) {
+      String fqdn = canonicalize(d.getIpAddr());
       if (scheme.equals("http")) {
       if (scheme.equals("http")) {
-        return d.getInfoAddr();
+        return fqdn + ":" + d.getInfoPort();
       } else if (scheme.equals("https")) {
       } else if (scheme.equals("https")) {
-        return d.getInfoSecureAddr();
+        return fqdn + ":" + d.getInfoSecurePort();
       } else {
       } else {
         throw new IllegalArgumentException("Unknown scheme:" + scheme);
         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.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
+import java.io.EOFException;
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 
 
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
@@ -67,7 +70,29 @@ public class BlockMetadataHeader {
     return checksum;
     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.
    * This reads all the fields till the beginning of checksum.
    * @param in 
    * @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.
  * The caching strategy we should use for an HDFS read or write operation.
  */
  */
 public class CachingStrategy {
 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() {
   public static CachingStrategy newDefaultStrategy() {
     return new CachingStrategy(null, null);
     return new CachingStrategy(null, null);
@@ -32,8 +32,28 @@ public class CachingStrategy {
     return new CachingStrategy(true, null);
     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) {
   public CachingStrategy(Boolean dropBehind, Long readahead) {
@@ -45,18 +65,10 @@ public class CachingStrategy {
     return dropBehind;
     return dropBehind;
   }
   }
   
   
-  public void setDropBehind(Boolean dropBehind) {
-    this.dropBehind = dropBehind;
-  }
-  
   public Long getReadahead() {
   public Long getReadahead() {
     return readahead;
     return readahead;
   }
   }
 
 
-  public void setReadahead(Long readahead) {
-    this.readahead = readahead;
-  }
-
   public String toString() {
   public String toString() {
     return "CachingStrategy(dropBehind=" + dropBehind +
     return "CachingStrategy(dropBehind=" + dropBehind +
         ", readahead=" + readahead + ")";
         ", 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.
    * Get the default chunk size.
    * @param conf the configuration
    * @param conf the configuration
@@ -228,7 +216,7 @@ public class DatanodeJspHelper {
       }
       }
     }
     }
     out.print("<br><a href=\"///"
     out.print("<br><a href=\"///"
-        + canonicalize(nnAddr) + ":"
+        + JspHelper.canonicalize(nnAddr) + ":"
         + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
         + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
     dfs.close();
     dfs.close();
   }
   }
@@ -359,7 +347,7 @@ public class DatanodeJspHelper {
     // generate a table and dump the info
     // generate a table and dump the info
     out.println("\n<table>");
     out.println("\n<table>");
     
     
-    String nnCanonicalName = canonicalize(nnAddr);
+    String nnCanonicalName = JspHelper.canonicalize(nnAddr);
     for (LocatedBlock cur : blocks) {
     for (LocatedBlock cur : blocks) {
       out.print("<tr>");
       out.print("<tr>");
       final String blockidstring = Long.toString(cur.getBlock().getBlockId());
       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 static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT;
 
 
 import java.io.DataInput;
 import java.io.DataInput;
-import java.io.DataOutput;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
@@ -45,13 +46,16 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirective;
 import org.apache.hadoop.hdfs.protocol.CacheDirective;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 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.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 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.StartupProgress.Counter;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
 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.security.AccessControlException;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -341,6 +345,67 @@ public final class CacheManager {
     return expiryTime;
     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
    * Get a CacheDirective by ID, validating the ID and that the directive
    * exists.
    * exists.
@@ -384,6 +449,15 @@ public final class CacheManager {
       directivesByPath.put(path, directives);
       directivesByPath.put(path, directives);
     }
     }
     directives.add(directive);
     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(
   public CacheDirectiveInfo addDirective(
-      CacheDirectiveInfo info, FSPermissionChecker pc)
+      CacheDirectiveInfo info, FSPermissionChecker pc, EnumSet<CacheFlag> flags)
       throws IOException {
       throws IOException {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
     CacheDirective directive;
     CacheDirective directive;
@@ -418,6 +492,14 @@ public final class CacheManager {
       short replication = validateReplication(info, (short)1);
       short replication = validateReplication(info, (short)1);
       long expiryTime = validateExpiryTime(info,
       long expiryTime = validateExpiryTime(info,
           CacheDirectiveInfo.Expiration.EXPIRY_NEVER);
           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
       // All validation passed
       // Add a new entry with the next available ID.
       // Add a new entry with the next available ID.
       long id = getNextDirectiveId();
       long id = getNextDirectiveId();
@@ -428,14 +510,11 @@ public final class CacheManager {
       throw e;
       throw e;
     }
     }
     LOG.info("addDirective of " + info + " successful.");
     LOG.info("addDirective of " + info + " successful.");
-    if (monitor != null) {
-      monitor.kick();
-    }
     return directive.toInfo();
     return directive.toInfo();
   }
   }
 
 
   public void modifyDirective(CacheDirectiveInfo info,
   public void modifyDirective(CacheDirectiveInfo info,
-      FSPermissionChecker pc) throws IOException {
+      FSPermissionChecker pc, EnumSet<CacheFlag> flags) throws IOException {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
     String idString =
     String idString =
         (info.getId() == null) ?
         (info.getId() == null) ?
@@ -463,6 +542,13 @@ public final class CacheManager {
       if (info.getPool() != null) {
       if (info.getPool() != null) {
         pool = getCachePool(validatePoolName(info));
         pool = getCachePool(validatePoolName(info));
         checkWritePermission(pc, pool);
         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);
       removeInternal(prevEntry);
       CacheDirective newEntry =
       CacheDirective newEntry =
@@ -489,9 +575,18 @@ public final class CacheManager {
     if (directives.size() == 0) {
     if (directives.size() == 0) {
       directivesByPath.remove(path);
       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());
     directivesById.remove(directive.getId());
-    directive.getPool().getDirectiveList().remove(directive);
+    pool.getDirectiveList().remove(directive);
     assert directive.getPool() == null;
     assert directive.getPool() == null;
+
+    if (monitor != null) {
+      monitor.setNeedsRescan();
+    }
   }
   }
 
 
   public void removeDirective(long id, FSPermissionChecker pc)
   public void removeDirective(long id, FSPermissionChecker pc)
@@ -505,9 +600,6 @@ public final class CacheManager {
       LOG.warn("removeDirective of " + id + " failed: ", e);
       LOG.warn("removeDirective of " + id + " failed: ", e);
       throw e;
       throw e;
     }
     }
-    if (monitor != null) {
-      monitor.kick();
-    }
     LOG.info("removeDirective of " + id + " successful.");
     LOG.info("removeDirective of " + id + " successful.");
   }
   }
 
 
@@ -527,6 +619,9 @@ public final class CacheManager {
     if (filter.getReplication() != null) {
     if (filter.getReplication() != null) {
       throw new IOException("Filtering by replication is unsupported.");
       throw new IOException("Filtering by replication is unsupported.");
     }
     }
+    if (monitor != null) {
+      monitor.waitForRescanIfNeeded();
+    }
     ArrayList<CacheDirectiveEntry> replies =
     ArrayList<CacheDirectiveEntry> replies =
         new ArrayList<CacheDirectiveEntry>(NUM_PRE_ALLOCATED_ENTRIES);
         new ArrayList<CacheDirectiveEntry>(NUM_PRE_ALLOCATED_ENTRIES);
     int numReplies = 0;
     int numReplies = 0;
@@ -573,16 +668,22 @@ public final class CacheManager {
   public CachePoolInfo addCachePool(CachePoolInfo info)
   public CachePoolInfo addCachePool(CachePoolInfo info)
       throws IOException {
       throws IOException {
     assert namesystem.hasWriteLock();
     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);
     return pool.getInfo(true);
   }
   }
 
 
@@ -597,42 +698,51 @@ public final class CacheManager {
   public void modifyCachePool(CachePoolInfo info)
   public void modifyCachePool(CachePoolInfo info)
       throws IOException {
       throws IOException {
     assert namesystem.hasWriteLock();
     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();
     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)
   public void removeCachePool(String poolName)
       throws IOException {
       throws IOException {
     assert namesystem.hasWriteLock();
     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>
   public BatchedListEntries<CachePoolEntry>
       listCachePools(FSPermissionChecker pc, String prevKey) {
       listCachePools(FSPermissionChecker pc, String prevKey) {
     assert namesystem.hasReadLock();
     assert namesystem.hasReadLock();
+    if (monitor != null) {
+      monitor.waitForRescanIfNeeded();
+    }
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     ArrayList<CachePoolEntry> results = 
     ArrayList<CachePoolEntry> results = 
         new ArrayList<CachePoolEntry>(NUM_PRE_ALLOCATED_ENTRIES);
         new ArrayList<CachePoolEntry>(NUM_PRE_ALLOCATED_ENTRIES);
@@ -782,7 +901,7 @@ public final class CacheManager {
    * @param sdPath path of the storage directory
    * @param sdPath path of the storage directory
    * @throws IOException
    * @throws IOException
    */
    */
-  public void saveState(DataOutput out, String sdPath)
+  public void saveState(DataOutputStream out, String sdPath)
       throws IOException {
       throws IOException {
     out.writeLong(nextDirectiveId);
     out.writeLong(nextDirectiveId);
     savePools(out, sdPath);
     savePools(out, sdPath);
@@ -805,7 +924,7 @@ public final class CacheManager {
   /**
   /**
    * Save cache pools to fsimage
    * Save cache pools to fsimage
    */
    */
-  private void savePools(DataOutput out,
+  private void savePools(DataOutputStream out,
       String sdPath) throws IOException {
       String sdPath) throws IOException {
     StartupProgress prog = NameNode.getStartupProgress();
     StartupProgress prog = NameNode.getStartupProgress();
     Step step = new Step(StepType.CACHE_POOLS, sdPath);
     Step step = new Step(StepType.CACHE_POOLS, sdPath);
@@ -814,7 +933,7 @@ public final class CacheManager {
     Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
     Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
     out.writeInt(cachePools.size());
     out.writeInt(cachePools.size());
     for (CachePool pool: cachePools.values()) {
     for (CachePool pool: cachePools.values()) {
-      pool.getInfo(true).writeTo(out);
+      FSImageSerialization.writeCachePoolInfo(out, pool.getInfo(true));
       counter.increment();
       counter.increment();
     }
     }
     prog.endStep(Phase.SAVING_CHECKPOINT, step);
     prog.endStep(Phase.SAVING_CHECKPOINT, step);
@@ -823,7 +942,7 @@ public final class CacheManager {
   /*
   /*
    * Save cache entries to fsimage
    * Save cache entries to fsimage
    */
    */
-  private void saveDirectives(DataOutput out, String sdPath)
+  private void saveDirectives(DataOutputStream out, String sdPath)
       throws IOException {
       throws IOException {
     StartupProgress prog = NameNode.getStartupProgress();
     StartupProgress prog = NameNode.getStartupProgress();
     Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
     Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
@@ -832,11 +951,7 @@ public final class CacheManager {
     Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
     Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
     out.writeInt(directivesById.size());
     out.writeInt(directivesById.size());
     for (CacheDirective directive : directivesById.values()) {
     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();
       counter.increment();
     }
     }
     prog.endStep(Phase.SAVING_CHECKPOINT, step);
     prog.endStep(Phase.SAVING_CHECKPOINT, step);
@@ -854,7 +969,7 @@ public final class CacheManager {
     prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfPools);
     prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfPools);
     Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
     Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
     for (int i = 0; i < numberOfPools; i++) {
     for (int i = 0; i < numberOfPools; i++) {
-      addCachePool(CachePoolInfo.readFrom(in));
+      addCachePool(FSImageSerialization.readCachePoolInfo(in));
       counter.increment();
       counter.increment();
     }
     }
     prog.endStep(Phase.LOADING_FSIMAGE, step);
     prog.endStep(Phase.LOADING_FSIMAGE, step);
@@ -871,19 +986,17 @@ public final class CacheManager {
     prog.setTotal(Phase.LOADING_FSIMAGE, step, numDirectives);
     prog.setTotal(Phase.LOADING_FSIMAGE, step, numDirectives);
     Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
     Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
     for (int i = 0; i < numDirectives; i++) {
     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
       // Get pool reference by looking it up in the map
+      final String poolName = info.getPool();
       CachePool pool = cachePools.get(poolName);
       CachePool pool = cachePools.get(poolName);
       if (pool == null) {
       if (pool == null) {
         throw new IOException("Directive refers to pool " + poolName +
         throw new IOException("Directive refers to pool " + poolName +
             ", which does not exist.");
             ", which does not exist.");
       }
       }
       CacheDirective directive =
       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);
       boolean addedDirective = pool.getDirectiveList().add(directive);
       assert addedDirective;
       assert addedDirective;
       if (directivesById.put(directive.getId(), directive) != null) {
       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 final class CachePool {
   public static final Log LOG = LogFactory.getLog(CachePool.class);
   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
   @Nonnull
   private final String poolName;
   private final String poolName;
 
 
@@ -71,7 +71,10 @@ public final class CachePool {
   @Nonnull
   @Nonnull
   private FsPermission mode;
   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 bytesNeeded;
   private long bytesCached;
   private long bytesCached;
@@ -118,10 +121,10 @@ public final class CachePool {
     }
     }
     FsPermission mode = (info.getMode() == null) ? 
     FsPermission mode = (info.getMode() == null) ? 
         FsPermission.getCachePoolDefault() : info.getMode();
         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(),
     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) {
   static CachePool createFromInfo(CachePoolInfo info) {
     return new CachePool(info.getPoolName(),
     return new CachePool(info.getPoolName(),
         info.getOwnerName(), info.getGroupName(),
         info.getOwnerName(), info.getGroupName(),
-        info.getMode(), info.getWeight());
+        info.getMode(), info.getLimit());
   }
   }
 
 
   CachePool(String poolName, String ownerName, String groupName,
   CachePool(String poolName, String ownerName, String groupName,
-      FsPermission mode, int weight) {
+      FsPermission mode, long limit) {
     Preconditions.checkNotNull(poolName);
     Preconditions.checkNotNull(poolName);
     Preconditions.checkNotNull(ownerName);
     Preconditions.checkNotNull(ownerName);
     Preconditions.checkNotNull(groupName);
     Preconditions.checkNotNull(groupName);
@@ -144,7 +147,7 @@ public final class CachePool {
     this.ownerName = ownerName;
     this.ownerName = ownerName;
     this.groupName = groupName;
     this.groupName = groupName;
     this.mode = new FsPermission(mode);
     this.mode = new FsPermission(mode);
-    this.weight = weight;
+    this.limit = limit;
   }
   }
 
 
   public String getPoolName() {
   public String getPoolName() {
@@ -177,16 +180,16 @@ public final class CachePool {
     this.mode = new FsPermission(mode);
     this.mode = new FsPermission(mode);
     return this;
     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;
     return this;
   }
   }
-  
+
   /**
   /**
    * Get either full or partial information about this CachePool.
    * Get either full or partial information about this CachePool.
    *
    *
@@ -204,7 +207,7 @@ public final class CachePool {
     return info.setOwnerName(ownerName).
     return info.setOwnerName(ownerName).
         setGroupName(groupName).
         setGroupName(groupName).
         setMode(new FsPermission(mode)).
         setMode(new FsPermission(mode)).
-        setWeight(weight);
+        setLimit(limit);
   }
   }
 
 
   /**
   /**
@@ -241,6 +244,10 @@ public final class CachePool {
     return bytesCached;
     return bytesCached;
   }
   }
 
 
+  public long getBytesOverlimit() {
+    return Math.max(bytesNeeded-limit, 0);
+  }
+
   public long getFilesNeeded() {
   public long getFilesNeeded() {
     return filesNeeded;
     return filesNeeded;
   }
   }
@@ -258,6 +265,7 @@ public final class CachePool {
     return new CachePoolStats.Builder().
     return new CachePoolStats.Builder().
         setBytesNeeded(bytesNeeded).
         setBytesNeeded(bytesNeeded).
         setBytesCached(bytesCached).
         setBytesCached(bytesCached).
+        setBytesOverlimit(getBytesOverlimit()).
         setFilesNeeded(filesNeeded).
         setFilesNeeded(filesNeeded).
         setFilesCached(filesCached).
         setFilesCached(filesCached).
         build();
         build();
@@ -291,7 +299,7 @@ public final class CachePool {
         append(", ownerName:").append(ownerName).
         append(", ownerName:").append(ownerName).
         append(", groupName:").append(groupName).
         append(", groupName:").append(groupName).
         append(", mode:").append(mode).
         append(", mode:").append(mode).
-        append(", weight:").append(weight).
+        append(", limit:").append(limit).
         append(" }").toString();
         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 long totalTimeTransactions;  // total time for all transactions
   private NameNodeMetrics metrics;
   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 =
   private ThreadLocal<OpInstanceCache> cache =
       new ThreadLocal<OpInstanceCache>() {
       new ThreadLocal<OpInstanceCache>() {
@@ -176,7 +176,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
   /**
    * The edit directories that are shared between primary and secondary.
    * The edit directories that are shared between primary and secondary.
    */
    */
-  private List<URI> sharedEditsDirs;
+  private final List<URI> sharedEditsDirs;
 
 
   private static class TransactionId {
   private static class TransactionId {
     public long txid;
     public long txid;
@@ -203,10 +203,6 @@ public class FSEditLog implements LogsPurgeable {
    * @param editsDirs List of journals to use
    * @param editsDirs List of journals to use
    */
    */
   FSEditLog(Configuration conf, NNStorage storage, List<URI> editsDirs) {
   FSEditLog(Configuration conf, NNStorage storage, List<URI> editsDirs) {
-    init(conf, storage, editsDirs);
-  }
-  
-  private void init(Configuration conf, NNStorage storage, List<URI> editsDirs) {
     isSyncRunning = false;
     isSyncRunning = false;
     this.conf = conf;
     this.conf = conf;
     this.storage = storage;
     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.io.InputStream;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.EnumMap;
 import java.util.EnumMap;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -650,7 +652,7 @@ public class FSEditLogLoader {
       ModifyCacheDirectiveInfoOp modifyOp =
       ModifyCacheDirectiveInfoOp modifyOp =
           (ModifyCacheDirectiveInfoOp) op;
           (ModifyCacheDirectiveInfoOp) op;
       fsNamesys.getCacheManager().modifyDirective(
       fsNamesys.getCacheManager().modifyDirective(
-          modifyOp.directive, null);
+          modifyOp.directive, null, EnumSet.of(CacheFlag.FORCE));
       if (toAddRetryCache) {
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
         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.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
-import java.util.Date;
 import java.util.EnumMap;
 import java.util.EnumMap;
 import java.util.List;
 import java.util.List;
 import java.util.zip.CheckedInputStream;
 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.classification.InterfaceStability;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.Options.Rename;
 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.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -120,11 +118,10 @@ import com.google.common.base.Preconditions;
 @InterfaceStability.Unstable
 @InterfaceStability.Unstable
 public abstract class FSEditLogOp {
 public abstract class FSEditLogOp {
   public final FSEditLogOpCodes opCode;
   public final FSEditLogOpCodes opCode;
-  long txid;
+  long txid = HdfsConstants.INVALID_TXID;
   byte[] rpcClientId = RpcConstants.DUMMY_CLIENT_ID;
   byte[] rpcClientId = RpcConstants.DUMMY_CLIENT_ID;
   int rpcCallId = RpcConstants.INVALID_CALL_ID;
   int rpcCallId = RpcConstants.INVALID_CALL_ID;
 
 
-  @SuppressWarnings("deprecation")
   final public static class OpInstanceCache {
   final public static class OpInstanceCache {
     private EnumMap<FSEditLogOpCodes, FSEditLogOp> inst = 
     private EnumMap<FSEditLogOpCodes, FSEditLogOp> inst = 
         new EnumMap<FSEditLogOpCodes, FSEditLogOp>(FSEditLogOpCodes.class);
         new EnumMap<FSEditLogOpCodes, FSEditLogOp>(FSEditLogOpCodes.class);
@@ -149,13 +146,10 @@ public abstract class FSEditLogOp {
       inst.put(OP_REASSIGN_LEASE, new ReassignLeaseOp());
       inst.put(OP_REASSIGN_LEASE, new ReassignLeaseOp());
       inst.put(OP_GET_DELEGATION_TOKEN, new GetDelegationTokenOp());
       inst.put(OP_GET_DELEGATION_TOKEN, new GetDelegationTokenOp());
       inst.put(OP_RENEW_DELEGATION_TOKEN, new RenewDelegationTokenOp());
       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_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_UPDATE_BLOCKS, new UpdateBlocksOp());
 
 
       inst.put(OP_ALLOW_SNAPSHOT, new AllowSnapshotOp());
       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_RENAME_SNAPSHOT, new RenameSnapshotOp());
       inst.put(OP_SET_GENSTAMP_V2, new SetGenstampV2Op());
       inst.put(OP_SET_GENSTAMP_V2, new SetGenstampV2Op());
       inst.put(OP_ALLOCATE_BLOCK_ID, new AllocateBlockIdOp());
       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_ADD_CACHE_POOL, new AddCachePoolOp());
       inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
       inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
       inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
       inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
@@ -187,7 +179,6 @@ public abstract class FSEditLogOp {
    */
    */
   private FSEditLogOp(FSEditLogOpCodes opCode) {
   private FSEditLogOp(FSEditLogOpCodes opCode) {
     this.opCode = opCode;
     this.opCode = opCode;
-    this.txid = HdfsConstants.INVALID_TXID;
   }
   }
 
 
   public long getTransactionId() {
   public long getTransactionId() {
@@ -2895,56 +2886,25 @@ public abstract class FSEditLogOp {
 
 
     @Override
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
     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);
       readRpcIds(in, logVersion);
     }
     }
 
 
     @Override
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
     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);
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
     }
 
 
     @Override
     @Override
     protected void toXml(ContentHandler contentHandler) throws SAXException {
     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);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
     }
     }
 
 
     @Override
     @Override
     void fromXml(Stanza st) throws InvalidXmlException {
     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);
       readRpcIdsFromXml(st);
     }
     }
 
 
@@ -2988,104 +2948,25 @@ public abstract class FSEditLogOp {
 
 
     @Override
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
     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);
       readRpcIds(in, logVersion);
     }
     }
 
 
     @Override
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
     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);
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
     }
 
 
     @Override
     @Override
     protected void toXml(ContentHandler contentHandler) throws SAXException {
     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);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
     }
     }
 
 
     @Override
     @Override
     void fromXml(Stanza st) throws InvalidXmlException {
     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);
       readRpcIdsFromXml(st);
     }
     }
 
 
@@ -3184,30 +3065,35 @@ public abstract class FSEditLogOp {
 
 
     public AddCachePoolOp setPool(CachePoolInfo info) {
     public AddCachePoolOp setPool(CachePoolInfo info) {
       this.info = 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;
       return this;
     }
     }
 
 
     @Override
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
     void readFields(DataInputStream in, int logVersion) throws IOException {
-      info = CachePoolInfo.readFrom(in);
+      info = FSImageSerialization.readCachePoolInfo(in);
       readRpcIds(in, logVersion);
       readRpcIds(in, logVersion);
     }
     }
 
 
     @Override
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
     public void writeFields(DataOutputStream out) throws IOException {
-      info.writeTo(out);
+      FSImageSerialization.writeCachePoolInfo(out, info);
       writeRpcIds(rpcClientId, rpcCallId, out);
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
     }
 
 
     @Override
     @Override
     protected void toXml(ContentHandler contentHandler) throws SAXException {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
-      info.writeXmlTo(contentHandler);
+      FSImageSerialization.writeCachePoolInfo(contentHandler, info);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
     }
     }
 
 
     @Override
     @Override
     void fromXml(Stanza st) throws InvalidXmlException {
     void fromXml(Stanza st) throws InvalidXmlException {
-      this.info = CachePoolInfo.readXmlFrom(st);
+      this.info = FSImageSerialization.readCachePoolInfo(st);
       readRpcIdsFromXml(st);
       readRpcIdsFromXml(st);
     }
     }
 
 
@@ -3219,7 +3105,7 @@ public abstract class FSEditLogOp {
       builder.append("ownerName=" + info.getOwnerName() + ",");
       builder.append("ownerName=" + info.getOwnerName() + ",");
       builder.append("groupName=" + info.getGroupName() + ",");
       builder.append("groupName=" + info.getGroupName() + ",");
       builder.append("mode=" + Short.toString(info.getMode().toShort()) + ",");
       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);
       appendRpcIdsToString(builder, rpcClientId, rpcCallId);
       builder.append("]");
       builder.append("]");
       return builder.toString();
       return builder.toString();
@@ -3245,25 +3131,25 @@ public abstract class FSEditLogOp {
 
 
     @Override
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
     void readFields(DataInputStream in, int logVersion) throws IOException {
-      info = CachePoolInfo.readFrom(in);
+      info = FSImageSerialization.readCachePoolInfo(in);
       readRpcIds(in, logVersion);
       readRpcIds(in, logVersion);
     }
     }
 
 
     @Override
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
     public void writeFields(DataOutputStream out) throws IOException {
-      info.writeTo(out);
+      FSImageSerialization.writeCachePoolInfo(out, info);
       writeRpcIds(rpcClientId, rpcCallId, out);
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
     }
 
 
     @Override
     @Override
     protected void toXml(ContentHandler contentHandler) throws SAXException {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
-      cachePoolInfoToXml(contentHandler, info);
+      FSImageSerialization.writeCachePoolInfo(contentHandler, info);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
     }
     }
 
 
     @Override
     @Override
     void fromXml(Stanza st) throws InvalidXmlException {
     void fromXml(Stanza st) throws InvalidXmlException {
-      this.info = cachePoolInfoFromXml(st);
+      this.info = FSImageSerialization.readCachePoolInfo(st);
       readRpcIdsFromXml(st);
       readRpcIdsFromXml(st);
     }
     }
 
 
@@ -3284,8 +3170,8 @@ public abstract class FSEditLogOp {
       if (info.getMode() != null) {
       if (info.getMode() != null) {
         fields.add("mode=" + info.getMode().toString());
         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));
       builder.append(Joiner.on(",").join(fields));
       appendRpcIdsToString(builder, rpcClientId, rpcCallId);
       appendRpcIdsToString(builder, rpcClientId, rpcCallId);
@@ -3439,9 +3325,7 @@ public abstract class FSEditLogOp {
      * @param in The stream to read from.
      * @param in The stream to read from.
      * @param logVersion The version of the data coming from the stream.
      * @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;
       this.logVersion = logVersion;
       if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
       if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
         this.checksum = new PureJavaCrc32();
         this.checksum = new PureJavaCrc32();
@@ -3757,41 +3641,4 @@ public abstract class FSEditLogOp {
     short mode = Short.valueOf(st.getValue("MODE"));
     short mode = Short.valueOf(st.getValue("MODE"));
     return new FsPermission(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;
 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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 
 
@@ -30,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Unstable
 @InterfaceStability.Unstable
 public enum FSEditLogOpCodes {
 public enum FSEditLogOpCodes {
   // last op code in file
   // last op code in file
-  OP_INVALID                    ((byte) -1),
   OP_ADD                        ((byte)  0),
   OP_ADD                        ((byte)  0),
   OP_RENAME_OLD                 ((byte)  1), // deprecated operation
   OP_RENAME_OLD                 ((byte)  1), // deprecated operation
   OP_DELETE                     ((byte)  2),
   OP_DELETE                     ((byte)  2),
@@ -69,9 +65,12 @@ public enum FSEditLogOpCodes {
   OP_ADD_CACHE_POOL                       ((byte) 35),
   OP_ADD_CACHE_POOL                       ((byte) 35),
   OP_MODIFY_CACHE_POOL                    ((byte) 36),
   OP_MODIFY_CACHE_POOL                    ((byte) 36),
   OP_REMOVE_CACHE_POOL                    ((byte) 37),
   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
    * Constructor
@@ -91,14 +90,7 @@ public enum FSEditLogOpCodes {
     return opCode;
     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
    * Converts byte to FSEditLogOpCodes enum value
@@ -107,6 +99,12 @@ public enum FSEditLogOpCodes {
    * @return enum with byte value of opCode
    * @return enum with byte value of opCode
    */
    */
   public static FSEditLogOpCodes fromByte(byte 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.DFSUtil;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.protocol.Block;
 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;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 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.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 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.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 
 
@@ -476,4 +483,202 @@ public class FSImageSerialization {
     }
     }
     return ret;
     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.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 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);
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = isPermissionEnabled ?
     final FSPermissionChecker pc = isPermissionEnabled ?
         getPermissionChecker() : null;
         getPermissionChecker() : null;
@@ -7078,7 +7079,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             "for this operation.");
             "for this operation.");
       }
       }
       CacheDirectiveInfo effectiveDirective = 
       CacheDirectiveInfo effectiveDirective = 
-          cacheManager.addDirective(directive, pc);
+          cacheManager.addDirective(directive, pc, flags);
       getEditLog().logAddCacheDirectiveInfo(effectiveDirective,
       getEditLog().logAddCacheDirectiveInfo(effectiveDirective,
           cacheEntry != null);
           cacheEntry != null);
       result = effectiveDirective.getId();
       result = effectiveDirective.getId();
@@ -7096,8 +7097,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return result;
     return result;
   }
   }
 
 
-  void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = isPermissionEnabled ?
     final FSPermissionChecker pc = isPermissionEnabled ?
         getPermissionChecker() : null;
         getPermissionChecker() : null;
@@ -7113,7 +7114,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new SafeModeException(
         throw new SafeModeException(
             "Cannot add cache directive", safeMode);
             "Cannot add cache directive", safeMode);
       }
       }
-      cacheManager.modifyDirective(directive, pc);
+      cacheManager.modifyDirective(directive, pc, flags);
       getEditLog().logModifyCacheDirectiveInfo(directive,
       getEditLog().logModifyCacheDirectiveInfo(directive,
           cacheEntry != null);
           cacheEntry != null);
       success = true;
       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;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import static org.apache.hadoop.util.Time.now;
+
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.*;
 import java.util.*;
 import java.io.*;
 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.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 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.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
@@ -88,6 +91,7 @@ public class GetImageServlet extends HttpServlet {
       final GetImageParams parsedParams = new GetImageParams(request, response);
       final GetImageParams parsedParams = new GetImageParams(request, response);
       final Configuration conf = (Configuration) context
       final Configuration conf = (Configuration) context
           .getAttribute(JspHelper.CURRENT_CONF);
           .getAttribute(JspHelper.CURRENT_CONF);
+      final NameNodeMetrics metrics = NameNode.getNameNodeMetrics();
       
       
       if (UserGroupInformation.isSecurityEnabled() && 
       if (UserGroupInformation.isSecurityEnabled() && 
           !isValidRequestor(context, request.getUserPrincipal().getName(), conf)) {
           !isValidRequestor(context, request.getUserPrincipal().getName(), conf)) {
@@ -128,14 +132,26 @@ public class GetImageServlet extends HttpServlet {
               throw new IOException(errorMessage);
               throw new IOException(errorMessage);
             }
             }
             CheckpointFaultInjector.getInstance().beforeGetImageSetsHeaders();
             CheckpointFaultInjector.getInstance().beforeGetImageSetsHeaders();
+            long start = now();
             serveFile(imageFile);
             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()) {
           } else if (parsedParams.isGetEdit()) {
             long startTxId = parsedParams.getStartTxId();
             long startTxId = parsedParams.getStartTxId();
             long endTxId = parsedParams.getEndTxId();
             long endTxId = parsedParams.getEndTxId();
             
             
             File editFile = nnImage.getStorage()
             File editFile = nnImage.getStorage()
                 .findFinalizedEditsFile(startTxId, endTxId);
                 .findFinalizedEditsFile(startTxId, endTxId);
+            long start = now();
             serveFile(editFile);
             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()) {
           } else if (parsedParams.isPutImage()) {
             final long txid = parsedParams.getTxId();
             final long txid = parsedParams.getTxId();
 
 
@@ -159,12 +175,18 @@ public class GetImageServlet extends HttpServlet {
                 UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
                 UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
               }
               }
               
               
+              long start = now();
               // issue a HTTP get request to download the new fsimage 
               // issue a HTTP get request to download the new fsimage 
               MD5Hash downloadImageDigest =
               MD5Hash downloadImageDigest =
                 TransferFsImage.downloadImageToStorage(
                 TransferFsImage.downloadImageToStorage(
                         parsedParams.getInfoServer(conf), txid,
                         parsedParams.getInfoServer(conf), txid,
                         nnImage.getStorage(), true);
                         nnImage.getStorage(), true);
               nnImage.saveDigestAndRenameCheckpointImage(txid, downloadImageDigest);
               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
               // Now that we have a new checkpoint, we might be able to
               // remove some old ones.
               // 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.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Set;
 import java.util.Set;
@@ -36,6 +37,7 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
@@ -1241,14 +1243,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
   @Override
   @Override
   public long addCacheDirective(
   public long addCacheDirective(
-      CacheDirectiveInfo path) throws IOException {
-    return namesystem.addCacheDirective(path);
+      CacheDirectiveInfo path, EnumSet<CacheFlag> flags) throws IOException {
+    return namesystem.addCacheDirective(path, flags);
   }
   }
 
 
   @Override
   @Override
   public void modifyCacheDirective(
   public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
-    namesystem.modifyCacheDirective(directive);
+      CacheDirectiveInfo directive, EnumSet<CacheFlag> flags) throws IOException {
+    namesystem.modifyCacheDirective(directive, flags);
   }
   }
 
 
   @Override
   @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")
   @Metric("Time loading FS Image at startup in msec")
   MutableGaugeInt fsImageLoadTime;
   MutableGaugeInt fsImageLoadTime;
 
 
+  @Metric("GetImageServlet getEdit")
+  MutableRate getEdit;
+  @Metric("GetImageServlet getImage")
+  MutableRate getImage;
+  @Metric("GetImageServlet putImage")
+  MutableRate putImage;
+
   NameNodeMetrics(String processName, String sessionId, int[] intervals) {
   NameNodeMetrics(String processName, String sessionId, int[] intervals) {
     registry.tag(ProcessName, processName).tag(SessionId, sessionId);
     registry.tag(ProcessName, processName).tag(SessionId, sessionId);
     
     
@@ -251,4 +258,16 @@ public class NameNodeMetrics {
   public void setSafeModeTime(long elapsed) {
   public void setSafeModeTime(long elapsed) {
     safeModeTime.set((int) 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;
 package org.apache.hadoop.hdfs.tools;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 
 
@@ -25,6 +26,7 @@ import org.apache.commons.lang.WordUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
@@ -135,6 +137,7 @@ public class CacheAdmin extends Configured implements Tool {
     public String getShortUsage() {
     public String getShortUsage() {
       return "[" + getName() +
       return "[" + getName() +
           " -path <path> -pool <pool-name> " +
           " -path <path> -pool <pool-name> " +
+          "[-force] " +
           "[-replication <replication>] [-ttl <time-to-live>]]\n";
           "[-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 " +
       listing.addRow("<pool-name>", "The pool to which the directive will be " +
           "added. You must have write permission on the cache pool "
           "added. You must have write permission on the cache pool "
           + "in order to add new directives.");
           + "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. " +
       listing.addRow("<replication>", "The cache replication factor to use. " +
           "Defaults to 1.");
           "Defaults to 1.");
       listing.addRow("<time-to-live>", "How long the directive is " +
       listing.addRow("<time-to-live>", "How long the directive is " +
@@ -174,7 +179,7 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
         return 1;
       }
       }
       builder.setPool(poolName);
       builder.setPool(poolName);
-
+      boolean force = StringUtils.popOption("-force", args);
       String replicationString =
       String replicationString =
           StringUtils.popOptionWithArgument("-replication", args);
           StringUtils.popOptionWithArgument("-replication", args);
       if (replicationString != null) {
       if (replicationString != null) {
@@ -201,8 +206,12 @@ public class CacheAdmin extends Configured implements Tool {
         
         
       DistributedFileSystem dfs = getDFS(conf);
       DistributedFileSystem dfs = getDFS(conf);
       CacheDirectiveInfo directive = builder.build();
       CacheDirectiveInfo directive = builder.build();
+      EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
+      if (force) {
+        flags.add(CacheFlag.FORCE);
+      }
       try {
       try {
-        long id = dfs.addCacheDirective(directive);
+        long id = dfs.addCacheDirective(directive, flags);
         System.out.println("Added cache directive " + id);
         System.out.println("Added cache directive " + id);
       } catch (IOException e) {
       } catch (IOException e) {
         System.err.println(prettifyException(e));
         System.err.println(prettifyException(e));
@@ -282,7 +291,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     @Override
     public String getShortUsage() {
     public String getShortUsage() {
       return "[" + getName() +
       return "[" + getName() +
-          " -id <id> [-path <path>] [-replication <replication>] " +
+          " -id <id> [-path <path>] [-force] [-replication <replication>] " +
           "[-pool <pool-name>] [-ttl <time-to-live>]]\n";
           "[-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("<id>", "The ID of the directive to modify (required)");
       listing.addRow("<path>", "A path to cache. The path can be " +
       listing.addRow("<path>", "A path to cache. The path can be " +
           "a directory or a file. (optional)");
           "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. " +
       listing.addRow("<replication>", "The cache replication factor to use. " +
           "(optional)");
           "(optional)");
       listing.addRow("<pool-name>", "The pool to which the directive will be " +
       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));
         builder.setPath(new Path(path));
         modified = true;
         modified = true;
       }
       }
+      boolean force = StringUtils.popOption("-force", args);
       String replicationString =
       String replicationString =
         StringUtils.popOptionWithArgument("-replication", args);
         StringUtils.popOptionWithArgument("-replication", args);
       if (replicationString != null) {
       if (replicationString != null) {
@@ -357,8 +369,12 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
         return 1;
       }
       }
       DistributedFileSystem dfs = getDFS(conf);
       DistributedFileSystem dfs = getDFS(conf);
+      EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
+      if (force) {
+        flags.add(CacheFlag.FORCE);
+      }
       try {
       try {
-        dfs.modifyCacheDirective(builder.build());
+        dfs.modifyCacheDirective(builder.build(), flags);
         System.out.println("Modified cache directive " + idString);
         System.out.println("Modified cache directive " + idString);
       } catch (IOException e) {
       } catch (IOException e) {
         System.err.println(prettifyException(e));
         System.err.println(prettifyException(e));
@@ -536,7 +552,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     @Override
     public String getShortUsage() {
     public String getShortUsage() {
       return "[" + NAME + " <name> [-owner <owner>] " +
       return "[" + NAME + " <name> [-owner <owner>] " +
-          "[-group <group>] [-mode <mode>] [-weight <weight>]]\n";
+          "[-group <group>] [-mode <mode>] [-limit <limit>]]\n";
     }
     }
 
 
     @Override
     @Override
@@ -551,11 +567,10 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<mode>", "UNIX-style permissions for the pool. " +
       listing.addRow("<mode>", "UNIX-style permissions for the pool. " +
           "Permissions are specified in octal, e.g. 0755. " +
           "Permissions are specified in octal, e.g. 0755. " +
           "By default, this is set to " + String.format("0%03o",
           "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" +
       return getShortUsage() + "\n" +
           "Add a new cache pool.\n\n" + 
           "Add a new cache pool.\n\n" + 
@@ -564,33 +579,31 @@ public class CacheAdmin extends Configured implements Tool {
 
 
     @Override
     @Override
     public int run(Configuration conf, List<String> args) throws IOException {
     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);
       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);
       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);
       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()) {
       if (!args.isEmpty()) {
         System.err.print("Can't understand arguments: " +
         System.err.print("Can't understand arguments: " +
@@ -599,11 +612,6 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
         return 1;
       }
       }
       DistributedFileSystem dfs = getDFS(conf);
       DistributedFileSystem dfs = getDFS(conf);
-      CachePoolInfo info = new CachePoolInfo(name).
-          setOwnerName(owner).
-          setGroupName(group).
-          setMode(new FsPermission((short)mode)).
-          setWeight(weight);
       try {
       try {
         dfs.addCachePool(info);
         dfs.addCachePool(info);
       } catch (IOException e) {
       } catch (IOException e) {
@@ -624,7 +632,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     @Override
     public String getShortUsage() {
     public String getShortUsage() {
       return "[" + getName() + " <name> [-owner <owner>] " +
       return "[" + getName() + " <name> [-owner <owner>] " +
-          "[-group <group>] [-mode <mode>] [-weight <weight>]]\n";
+          "[-group <group>] [-mode <mode>] [-limit <limit>]]\n";
     }
     }
 
 
     @Override
     @Override
@@ -635,11 +643,12 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<owner>", "Username of the owner of the pool");
       listing.addRow("<owner>", "Username of the owner of the pool");
       listing.addRow("<group>", "Groupname of the group 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("<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" +
       return getShortUsage() + "\n" +
           WordUtils.wrap("Modifies the metadata of an existing cache pool. " +
           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" +
           MAX_LINE_WIDTH) + "\n\n" +
           listing.toString();
           listing.toString();
     }
     }
@@ -651,9 +660,9 @@ public class CacheAdmin extends Configured implements Tool {
       String modeString = StringUtils.popOptionWithArgument("-mode", args);
       String modeString = StringUtils.popOptionWithArgument("-mode", args);
       Integer mode = (modeString == null) ?
       Integer mode = (modeString == null) ?
           null : Integer.parseInt(modeString, 8);
           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);
       String name = StringUtils.popFirstNonOption(args);
       if (name == null) {
       if (name == null) {
         System.err.println("You must specify a name when creating a " +
         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()));
         info.setMode(new FsPermission(mode.shortValue()));
         changed = true;
         changed = true;
       }
       }
-      if (weight != null) {
-        info.setWeight(weight);
+      if (limit != null) {
+        info.setLimit(limit);
         changed = true;
         changed = true;
       }
       }
       if (!changed) {
       if (!changed) {
@@ -709,8 +718,8 @@ public class CacheAdmin extends Configured implements Tool {
         System.out.print(prefix + "mode " + new FsPermission(mode.shortValue()));
         System.out.print(prefix + "mode " + new FsPermission(mode.shortValue()));
         prefix = " and ";
         prefix = " and ";
       }
       }
-      if (weight != null) {
-        System.out.print(prefix + "weight " + weight);
+      if (limit != null) {
+        System.out.print(prefix + "limit " + limit);
         prefix = " and ";
         prefix = " and ";
       }
       }
       System.out.print("\n");
       System.out.print("\n");
@@ -804,11 +813,12 @@ public class CacheAdmin extends Configured implements Tool {
           addField("OWNER", Justification.LEFT).
           addField("OWNER", Justification.LEFT).
           addField("GROUP", Justification.LEFT).
           addField("GROUP", Justification.LEFT).
           addField("MODE", Justification.LEFT).
           addField("MODE", Justification.LEFT).
-          addField("WEIGHT", Justification.RIGHT);
+          addField("LIMIT", Justification.RIGHT);
       if (printStats) {
       if (printStats) {
         builder.
         builder.
             addField("BYTES_NEEDED", Justification.RIGHT).
             addField("BYTES_NEEDED", Justification.RIGHT).
             addField("BYTES_CACHED", Justification.RIGHT).
             addField("BYTES_CACHED", Justification.RIGHT).
+            addField("BYTES_OVERLIMIT", Justification.RIGHT).
             addField("FILES_NEEDED", Justification.RIGHT).
             addField("FILES_NEEDED", Justification.RIGHT).
             addField("FILES_CACHED", 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.getOwnerName());
             row.add(info.getGroupName());
             row.add(info.getGroupName());
             row.add(info.getMode() != null ? info.getMode().toString() : null);
             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) {
             if (printStats) {
               CachePoolStats stats = entry.getStats();
               CachePoolStats stats = entry.getStats();
               row.add(Long.toString(stats.getBytesNeeded()));
               row.add(Long.toString(stats.getBytesNeeded()));
               row.add(Long.toString(stats.getBytesCached()));
               row.add(Long.toString(stats.getBytesCached()));
+              row.add(Long.toString(stats.getBytesOverlimit()));
               row.add(Long.toString(stats.getFilesNeeded()));
               row.add(Long.toString(stats.getFilesNeeded()));
               row.add(Long.toString(stats.getFilesCached()));
               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;
   required bool hasExpired = 5;
 }
 }
 
 
+enum CacheFlagProto {
+  FORCE = 0x01;    // Ignore pool resource limits
+}
+
 message AddCacheDirectiveRequestProto {
 message AddCacheDirectiveRequestProto {
   required CacheDirectiveInfoProto info = 1;
   required CacheDirectiveInfoProto info = 1;
+  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
 }
 }
 
 
 message AddCacheDirectiveResponseProto {
 message AddCacheDirectiveResponseProto {
@@ -396,6 +401,7 @@ message AddCacheDirectiveResponseProto {
 
 
 message ModifyCacheDirectiveRequestProto {
 message ModifyCacheDirectiveRequestProto {
   required CacheDirectiveInfoProto info = 1;
   required CacheDirectiveInfoProto info = 1;
+  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
 }
 }
 
 
 message ModifyCacheDirectiveResponseProto {
 message ModifyCacheDirectiveResponseProto {
@@ -428,14 +434,15 @@ message CachePoolInfoProto {
   optional string ownerName = 2;
   optional string ownerName = 2;
   optional string groupName = 3;
   optional string groupName = 3;
   optional int32 mode = 4;
   optional int32 mode = 4;
-  optional int32 weight = 5;
+  optional int64 limit = 5;
 }
 }
 
 
 message CachePoolStatsProto {
 message CachePoolStatsProto {
   required int64 bytesNeeded = 1;
   required int64 bytesNeeded = 1;
   required int64 bytesCached = 2;
   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 {
 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>
   <name>dfs.client.cache.readahead</name>
   <value></value>
   <value></value>
   <description>
   <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
     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
     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 rather than a setting for the entire datanode.  If present, this
     setting will override the DataNode default.
     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
     If the native libraries are not available to the DataNode, this
     configuration has no effect.
     configuration has no effect.
   </description>
   </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.io.*;
 import java.net.*;
 import java.net.*;
+import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.*;
 import java.util.*;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeoutException;
@@ -346,9 +347,15 @@ public class DFSTestUtil {
         // Swallow exceptions
         // Swallow exceptions
       }
       }
       System.out.println("Waiting for "+corruptRepls+" corrupt replicas");
       System.out.println("Waiting for "+corruptRepls+" corrupt replicas");
-      repls = ns.getBlockManager().numCorruptReplicas(b.getLocalBlock());
       count++;
       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) {
     if (count == ATTEMPTS) {
       throw new TimeoutException("Timed out waiting for corrupt replicas."
       throw new TimeoutException("Timed out waiting for corrupt replicas."
@@ -1036,20 +1043,20 @@ public class DFSTestUtil {
     // OP_ADD_CACHE_POOL
     // OP_ADD_CACHE_POOL
     filesystem.addCachePool(new CachePoolInfo("pool1"));
     filesystem.addCachePool(new CachePoolInfo("pool1"));
     // OP_MODIFY_CACHE_POOL
     // OP_MODIFY_CACHE_POOL
-    filesystem.modifyCachePool(new CachePoolInfo("pool1").setWeight(99));
+    filesystem.modifyCachePool(new CachePoolInfo("pool1").setLimit(99l));
     // OP_ADD_PATH_BASED_CACHE_DIRECTIVE
     // OP_ADD_PATH_BASED_CACHE_DIRECTIVE
     long id = filesystem.addCacheDirective(
     long id = filesystem.addCacheDirective(
         new CacheDirectiveInfo.Builder().
         new CacheDirectiveInfo.Builder().
             setPath(new Path("/path")).
             setPath(new Path("/path")).
             setReplication((short)1).
             setReplication((short)1).
             setPool("pool1").
             setPool("pool1").
-            build());
+            build(), EnumSet.of(CacheFlag.FORCE));
     // OP_MODIFY_PATH_BASED_CACHE_DIRECTIVE
     // OP_MODIFY_PATH_BASED_CACHE_DIRECTIVE
     filesystem.modifyCacheDirective(
     filesystem.modifyCacheDirective(
         new CacheDirectiveInfo.Builder().
         new CacheDirectiveInfo.Builder().
             setId(id).
             setId(id).
             setReplication((short)2).
             setReplication((short)2).
-            build());
+            build(), EnumSet.of(CacheFlag.FORCE));
     // OP_REMOVE_PATH_BASED_CACHE_DIRECTIVE
     // OP_REMOVE_PATH_BASED_CACHE_DIRECTIVE
     filesystem.removeCacheDirective(id);
     filesystem.removeCacheDirective(id);
     // OP_REMOVE_CACHE_POOL
     // OP_REMOVE_CACHE_POOL
@@ -1059,4 +1066,10 @@ public class DFSTestUtil {
   public static void abortStream(DFSOutputStream out) throws IOException {
   public static void abortStream(DFSOutputStream out) throws IOException {
     out.abort();
     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.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 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)
     public void setup(File blockFile, boolean usingChecksums)
-        throws IOException;
+        throws IOException {
+      // default: no-op
+    }
     public void doTest(BlockReaderLocal reader, byte original[])
     public void doTest(BlockReaderLocal reader, byte original[])
-        throws IOException;
+        throws IOException {
+      // default: no-op
+    }
   }
   }
   
   
   public void runBlockReaderLocalTest(BlockReaderLocalTest test,
   public void runBlockReaderLocalTest(BlockReaderLocalTest test,
-      boolean checksum) throws IOException {
+      boolean checksum, long readahead) throws IOException {
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
     HdfsConfiguration conf = new HdfsConfiguration();
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.
     conf.setBoolean(DFSConfigKeys.
         DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, !checksum);
         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");
     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 Path TEST_PATH = new Path("/a");
     final long RANDOM_SEED = 4567L;
     final long RANDOM_SEED = 4567L;
     BlockReaderLocal blockReaderLocal = null;
     BlockReaderLocal blockReaderLocal = null;
@@ -143,45 +158,51 @@ public class TestBlockReaderLocal {
       cluster.shutdown();
       cluster.shutdown();
       cluster = null;
       cluster = null;
       test.setup(dataFile, checksum);
       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;
       dataIn = null;
-      checkIn = null;
+      metaIn = null;
       test.doTest(blockReaderLocal, original);
       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 {
     } finally {
       if (fsIn != null) fsIn.close();
       if (fsIn != null) fsIn.close();
       if (fs != null) fs.close();
       if (fs != null) fs.close();
       if (cluster != null) cluster.shutdown();
       if (cluster != null) cluster.shutdown();
       if (dataIn != null) dataIn.close();
       if (dataIn != null) dataIn.close();
-      if (checkIn != null) checkIn.close();
+      if (metaIn != null) metaIn.close();
       if (blockReaderLocal != null) blockReaderLocal.close();
       if (blockReaderLocal != null) blockReaderLocal.close();
     }
     }
   }
   }
   
   
   private static class TestBlockReaderLocalImmediateClose 
   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
   @Test
   public void testBlockReaderLocalImmediateClose() throws IOException {
   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 
   private static class TestBlockReaderSimpleReads 
-      implements BlockReaderLocalTest {
-    @Override
-    public void setup(File blockFile, boolean usingChecksums)
-        throws IOException { }
+      extends BlockReaderLocalTest {
     @Override
     @Override
     public void doTest(BlockReaderLocal reader, byte original[]) 
     public void doTest(BlockReaderLocal reader, byte original[]) 
         throws IOException {
         throws IOException {
@@ -194,24 +215,43 @@ public class TestBlockReaderLocal {
       assertArrayRegionsEqual(original, 1024, buf, 1024, 513);
       assertArrayRegionsEqual(original, 1024, buf, 1024, 513);
       reader.readFully(buf, 1537, 514);
       reader.readFully(buf, 1537, 514);
       assertArrayRegionsEqual(original, 1537, 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
   @Test
   public void testBlockReaderSimpleReads() throws IOException {
   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
   @Test
   public void testBlockReaderSimpleReadsNoChecksum() throws IOException {
   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 
   private static class TestBlockReaderLocalArrayReads2 
-      implements BlockReaderLocalTest {
-    @Override
-    public void setup(File blockFile, boolean usingChecksums)
-        throws IOException { }
+      extends BlockReaderLocalTest {
     @Override
     @Override
     public void doTest(BlockReaderLocal reader, byte original[]) 
     public void doTest(BlockReaderLocal reader, byte original[]) 
         throws IOException {
         throws IOException {
@@ -234,21 +274,30 @@ public class TestBlockReaderLocal {
   @Test
   @Test
   public void testBlockReaderLocalArrayReads2() throws IOException {
   public void testBlockReaderLocalArrayReads2() throws IOException {
     runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
     runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
-        true);
+        true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
   }
   }
 
 
   @Test
   @Test
   public void testBlockReaderLocalArrayReads2NoChecksum()
   public void testBlockReaderLocalArrayReads2NoChecksum()
       throws IOException {
       throws IOException {
     runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
     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 
   private static class TestBlockReaderLocalByteBufferReads 
-      implements BlockReaderLocalTest {
-    @Override
-    public void setup(File blockFile, boolean usingChecksums)
-        throws IOException { }
+      extends BlockReaderLocalTest {
     @Override
     @Override
     public void doTest(BlockReaderLocal reader, byte original[]) 
     public void doTest(BlockReaderLocal reader, byte original[]) 
         throws IOException {
         throws IOException {
@@ -268,19 +317,105 @@ public class TestBlockReaderLocal {
   @Test
   @Test
   public void testBlockReaderLocalByteBufferReads()
   public void testBlockReaderLocalByteBufferReads()
       throws IOException {
       throws IOException {
-    runBlockReaderLocalTest(
-        new TestBlockReaderLocalByteBufferReads(), true);
+    runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
+        true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
   }
   }
 
 
   @Test
   @Test
   public void testBlockReaderLocalByteBufferReadsNoChecksum()
   public void testBlockReaderLocalByteBufferReadsNoChecksum()
       throws IOException {
       throws IOException {
     runBlockReaderLocalTest(
     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
   private static class TestBlockReaderLocalReadCorruptStart
-      implements BlockReaderLocalTest {
+      extends BlockReaderLocalTest {
     boolean usingChecksums = false;
     boolean usingChecksums = false;
     @Override
     @Override
     public void setup(File blockFile, boolean usingChecksums)
     public void setup(File blockFile, boolean usingChecksums)
@@ -314,11 +449,12 @@ public class TestBlockReaderLocal {
   @Test
   @Test
   public void testBlockReaderLocalReadCorruptStart()
   public void testBlockReaderLocalReadCorruptStart()
       throws IOException {
       throws IOException {
-    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true);
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true,
+        DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
   }
   }
   
   
   private static class TestBlockReaderLocalReadCorrupt
   private static class TestBlockReaderLocalReadCorrupt
-      implements BlockReaderLocalTest {
+      extends BlockReaderLocalTest {
     boolean usingChecksums = false;
     boolean usingChecksums = false;
     @Override
     @Override
     public void setup(File blockFile, boolean usingChecksums) 
     public void setup(File blockFile, boolean usingChecksums) 
@@ -364,8 +500,136 @@ public class TestBlockReaderLocal {
   @Test
   @Test
   public void testBlockReaderLocalReadCorrupt()
   public void testBlockReaderLocalReadCorrupt()
       throws IOException {
       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)
   @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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 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.hdfs.net.Peer;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -138,7 +139,8 @@ public class TestConnCache {
                            Matchers.anyLong(),
                            Matchers.anyLong(),
                            Matchers.anyInt(),
                            Matchers.anyInt(),
                            Matchers.anyBoolean(),
                            Matchers.anyBoolean(),
-                           Matchers.anyString());
+                           Matchers.anyString(),
+                           (CachingStrategy)Matchers.anyObject());
 
 
     // Initial read
     // Initial read
     pread(in, 0, dataBuf, 0, dataBuf.length, authenticData);
     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);
     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
    * Tests recommission for non federated cluster
    */
    */
@@ -388,7 +440,20 @@ public class TestDecommission {
         DFSClient client = getDfsClient(cluster.getNameNode(i), conf);
         DFSClient client = getDfsClient(cluster.getNameNode(i), conf);
         assertEquals("All datanodes must be alive", numDatanodes, 
         assertEquals("All datanodes must be alive", numDatanodes, 
             client.datanodeReport(DatanodeReportType.LIVE).length);
             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);
         cleanupFile(fileSys, file1);
       }
       }
     }
     }
@@ -429,12 +494,25 @@ public class TestDecommission {
       DFSClient client = getDfsClient(cluster.getNameNode(i), conf);
       DFSClient client = getDfsClient(cluster.getNameNode(i), conf);
       assertEquals("All datanodes must be alive", numDatanodes, 
       assertEquals("All datanodes must be alive", numDatanodes, 
           client.datanodeReport(DatanodeReportType.LIVE).length);
           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
       // stop decommission and check if the new replicas are removed
       recomissionNode(decomNode);
       recomissionNode(decomNode);
       // wait for the block to be deleted
       // wait for the block to be deleted
-      int tries = 0;
+      tries = 0;
       while (tries++ < 20) {
       while (tries++ < 20) {
         try {
         try {
           Thread.sleep(1000);
           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)
       assertTrue("/ should be a directory", fs.getFileStatus(path)
           .isDirectory() == true);
           .isDirectory() == true);
       
       
-      // create a new file in home directory. Do not close it.
       byte[] fileData = AppendTestUtil.randomBytes(seed, size);
       byte[] fileData = AppendTestUtil.randomBytes(seed, size);
       Path file1 = fs.makeQualified(new Path("filelocal.dat"));
       Path file1 = fs.makeQualified(new Path("filelocal.dat"));
       FSDataOutputStream stm = createFile(fs, file1, 1);
       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")
         .setOwnerName("carlton")
         .setGroupName("party")
         .setGroupName("party")
         .setMode(new FsPermission((short)0700))
         .setMode(new FsPermission((short)0700))
-        .setWeight(1989));
+        .setLimit(1989l));
     // OP_ADD_PATH_BASED_CACHE_DIRECTIVE 33
     // OP_ADD_PATH_BASED_CACHE_DIRECTIVE 33
     long id = dfs.addCacheDirective(
     long id = dfs.addCacheDirective(
         new CacheDirectiveInfo.Builder().
         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.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.Date;
+import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
@@ -43,6 +44,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileSystemTestHelper;
@@ -92,25 +94,49 @@ public class TestCacheDirectives {
   static private MiniDFSCluster cluster;
   static private MiniDFSCluster cluster;
   static private DistributedFileSystem dfs;
   static private DistributedFileSystem dfs;
   static private NamenodeProtocols proto;
   static private NamenodeProtocols proto;
+  static private NameNode namenode;
   static private CacheManipulator prevCacheManipulator;
   static private CacheManipulator prevCacheManipulator;
 
 
   static {
   static {
+    NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
     EditLogFileOutputStream.setShouldSkipFsyncForTesting(false);
     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
     // 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_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();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
     dfs = cluster.getFileSystem();
     proto = cluster.getNameNodeRpc();
     proto = cluster.getNameNodeRpc();
+    namenode = cluster.getNameNode();
     prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
     prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
     NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
     NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
-    LogManager.getLogger(CacheReplicationMonitor.class).setLevel(Level.TRACE);
+    LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel(
+        Level.TRACE);
   }
   }
 
 
   @After
   @After
@@ -127,7 +153,7 @@ public class TestCacheDirectives {
     final String poolName = "pool1";
     final String poolName = "pool1";
     CachePoolInfo info = new CachePoolInfo(poolName).
     CachePoolInfo info = new CachePoolInfo(poolName).
         setOwnerName("bob").setGroupName("bobgroup").
         setOwnerName("bob").setGroupName("bobgroup").
-        setMode(new FsPermission((short)0755)).setWeight(150);
+        setMode(new FsPermission((short)0755)).setLimit(150l);
 
 
     // Add a pool
     // Add a pool
     dfs.addCachePool(info);
     dfs.addCachePool(info);
@@ -168,7 +194,7 @@ public class TestCacheDirectives {
 
 
     // Modify the pool
     // Modify the pool
     info.setOwnerName("jane").setGroupName("janegroup")
     info.setOwnerName("jane").setGroupName("janegroup")
-        .setMode(new FsPermission((short)0700)).setWeight(314);
+        .setMode(new FsPermission((short)0700)).setLimit(314l);
     dfs.modifyCachePool(info);
     dfs.modifyCachePool(info);
 
 
     // Do some invalid modify pools
     // Do some invalid modify pools
@@ -263,10 +289,10 @@ public class TestCacheDirectives {
     String ownerName = "abc";
     String ownerName = "abc";
     String groupName = "123";
     String groupName = "123";
     FsPermission mode = new FsPermission((short)0755);
     FsPermission mode = new FsPermission((short)0755);
-    int weight = 150;
+    long limit = 150;
     dfs.addCachePool(new CachePoolInfo(poolName).
     dfs.addCachePool(new CachePoolInfo(poolName).
         setOwnerName(ownerName).setGroupName(groupName).
         setOwnerName(ownerName).setGroupName(groupName).
-        setMode(mode).setWeight(weight));
+        setMode(mode).setLimit(limit));
     
     
     RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
     RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
     CachePoolInfo info = iter.next().getInfo();
     CachePoolInfo info = iter.next().getInfo();
@@ -277,10 +303,10 @@ public class TestCacheDirectives {
     ownerName = "def";
     ownerName = "def";
     groupName = "456";
     groupName = "456";
     mode = new FsPermission((short)0700);
     mode = new FsPermission((short)0700);
-    weight = 151;
+    limit = 151;
     dfs.modifyCachePool(new CachePoolInfo(poolName).
     dfs.modifyCachePool(new CachePoolInfo(poolName).
         setOwnerName(ownerName).setGroupName(groupName).
         setOwnerName(ownerName).setGroupName(groupName).
-        setMode(mode).setWeight(weight));
+        setMode(mode).setLimit(limit));
 
 
     iter = dfs.listCachePools();
     iter = dfs.listCachePools();
     info = iter.next().getInfo();
     info = iter.next().getInfo();
@@ -288,7 +314,7 @@ public class TestCacheDirectives {
     assertEquals(ownerName, info.getOwnerName());
     assertEquals(ownerName, info.getOwnerName());
     assertEquals(groupName, info.getGroupName());
     assertEquals(groupName, info.getGroupName());
     assertEquals(mode, info.getMode());
     assertEquals(mode, info.getMode());
-    assertEquals(Integer.valueOf(weight), info.getWeight());
+    assertEquals(limit, (long)info.getLimit());
 
 
     dfs.removeCachePool(poolName);
     dfs.removeCachePool(poolName);
     iter = dfs.listCachePools();
     iter = dfs.listCachePools();
@@ -495,30 +521,22 @@ public class TestCacheDirectives {
 
 
   @Test(timeout=60000)
   @Test(timeout=60000)
   public void testCacheManagerRestart() throws Exception {
   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
     // Create and validate a pool
     final String pool = "poolparty";
     final String pool = "poolparty";
     String groupName = "partygroup";
     String groupName = "partygroup";
     FsPermission mode = new FsPermission((short)0777);
     FsPermission mode = new FsPermission((short)0777);
-    int weight = 747;
+    long limit = 747;
     dfs.addCachePool(new CachePoolInfo(pool)
     dfs.addCachePool(new CachePoolInfo(pool)
         .setGroupName(groupName)
         .setGroupName(groupName)
         .setMode(mode)
         .setMode(mode)
-        .setWeight(weight));
+        .setLimit(limit));
     RemoteIterator<CachePoolEntry> pit = dfs.listCachePools();
     RemoteIterator<CachePoolEntry> pit = dfs.listCachePools();
     assertTrue("No cache pools found", pit.hasNext());
     assertTrue("No cache pools found", pit.hasNext());
     CachePoolInfo info = pit.next().getInfo();
     CachePoolInfo info = pit.next().getInfo();
     assertEquals(pool, info.getPoolName());
     assertEquals(pool, info.getPoolName());
     assertEquals(groupName, info.getGroupName());
     assertEquals(groupName, info.getGroupName());
     assertEquals(mode, info.getMode());
     assertEquals(mode, info.getMode());
-    assertEquals(weight, (int)info.getWeight());
+    assertEquals(limit, (long)info.getLimit());
     assertFalse("Unexpected # of cache pools found", pit.hasNext());
     assertFalse("Unexpected # of cache pools found", pit.hasNext());
   
   
     // Create some cache entries
     // Create some cache entries
@@ -556,7 +574,7 @@ public class TestCacheDirectives {
     assertEquals(pool, info.getPoolName());
     assertEquals(pool, info.getPoolName());
     assertEquals(groupName, info.getGroupName());
     assertEquals(groupName, info.getGroupName());
     assertEquals(mode, info.getMode());
     assertEquals(mode, info.getMode());
-    assertEquals(weight, (int)info.getWeight());
+    assertEquals(limit, (long)info.getLimit());
     assertFalse("Unexpected # of cache pools found", pit.hasNext());
     assertFalse("Unexpected # of cache pools found", pit.hasNext());
   
   
     dit = dfs.listCacheDirectives(null);
     dit = dfs.listCacheDirectives(null);
@@ -762,91 +780,64 @@ public class TestCacheDirectives {
         numCachedReplicas);
         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)
   @Test(timeout=120000)
   public void testWaitForCachedReplicas() throws Exception {
   public void testWaitForCachedReplicas() throws Exception {
-    HdfsConfiguration conf = createCachingConf();
     FileSystemTestHelper helper = new FileSystemTestHelper();
     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)
   @Test(timeout=120000)
   public void testAddingCacheDirectiveInfosWhenCachingIsDisabled()
   public void testAddingCacheDirectiveInfosWhenCachingIsDisabled()
       throws Exception {
       throws Exception {
+    cluster.shutdown();
     HdfsConfiguration conf = createCachingConf();
     HdfsConfiguration conf = createCachingConf();
     conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, false);
     conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, false);
     MiniDFSCluster cluster =
     MiniDFSCluster cluster =
@@ -894,103 +885,92 @@ public class TestCacheDirectives {
 
 
   @Test(timeout=120000)
   @Test(timeout=120000)
   public void testWaitForCachedReplicasInDirectory() throws Exception {
   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().
           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().
           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)
   @Test(timeout=120000)
   public void testReplicationFactor() throws Exception {
   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().
           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)
   @Test(timeout=60000)
@@ -1081,11 +1050,12 @@ public class TestCacheDirectives {
     assertNull("Unexpected owner name", info.getOwnerName());
     assertNull("Unexpected owner name", info.getOwnerName());
     assertNull("Unexpected group name", info.getGroupName());
     assertNull("Unexpected group name", info.getGroupName());
     assertNull("Unexpected mode", info.getMode());
     assertNull("Unexpected mode", info.getMode());
-    assertNull("Unexpected weight", info.getWeight());
+    assertNull("Unexpected limit", info.getLimit());
     // Modify the pool so myuser is now the owner
     // Modify the pool so myuser is now the owner
+    final long limit = 99;
     dfs.modifyCachePool(new CachePoolInfo(poolName)
     dfs.modifyCachePool(new CachePoolInfo(poolName)
         .setOwnerName(myUser.getShortUserName())
         .setOwnerName(myUser.getShortUserName())
-        .setWeight(99));
+        .setLimit(limit));
     // Should see full info
     // Should see full info
     it = myDfs.listCachePools();
     it = myDfs.listCachePools();
     info = it.next().getInfo();
     info = it.next().getInfo();
@@ -1096,60 +1066,127 @@ public class TestCacheDirectives {
     assertNotNull("Expected group name", info.getGroupName());
     assertNotNull("Expected group name", info.getGroupName());
     assertEquals("Mismatched mode", (short) 0700,
     assertEquals("Mismatched mode", (short) 0700,
         info.getMode().toShort());
         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 {
   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 {
     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)
       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.common.Util.fileAsURI;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil.assertNNHasCheckpoints;
 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.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.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 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.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
@@ -106,6 +110,7 @@ public class TestCheckpoint {
   }
   }
 
 
   static final Log LOG = LogFactory.getLog(TestCheckpoint.class); 
   static final Log LOG = LogFactory.getLog(TestCheckpoint.class); 
+  static final String NN_METRICS = "NameNodeActivity";
   
   
   static final long seed = 0xDEADBEEFL;
   static final long seed = 0xDEADBEEFL;
   static final int blockSize = 4096;
   static final int blockSize = 4096;
@@ -1048,6 +1053,14 @@ public class TestCheckpoint {
       //
       //
       secondary = startSecondaryNameNode(conf);
       secondary = startSecondaryNameNode(conf);
       secondary.doCheckpoint();
       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 {
     } finally {
       fileSys.close();
       fileSys.close();
       cleanup(secondary);
       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.IOException;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.SortedMap;
 
 
@@ -383,4 +384,33 @@ public class TestFSEditLogLoader {
     assertTrue(!validation.hasCorruptHeader());
     assertTrue(!validation.hasCorruptHeader());
     assertEquals(HdfsConstants.INVALID_TXID, validation.getEndTxId());
     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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
@@ -760,7 +761,7 @@ public class TestRetryCacheWithHA {
 
 
     @Override
     @Override
     void invoke() throws Exception {
     void invoke() throws Exception {
-      result = client.addCacheDirective(directive);
+      result = client.addCacheDirective(directive, EnumSet.of(CacheFlag.FORCE));
     }
     }
 
 
     @Override
     @Override
@@ -802,7 +803,7 @@ public class TestRetryCacheWithHA {
     @Override
     @Override
     void prepare() throws Exception {
     void prepare() throws Exception {
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
-      id = client.addCacheDirective(directive);
+      id = client.addCacheDirective(directive, EnumSet.of(CacheFlag.FORCE));
     }
     }
 
 
     @Override
     @Override
@@ -811,7 +812,7 @@ public class TestRetryCacheWithHA {
           new CacheDirectiveInfo.Builder().
           new CacheDirectiveInfo.Builder().
               setId(id).
               setId(id).
               setReplication(newReplication).
               setReplication(newReplication).
-              build());
+              build(), EnumSet.of(CacheFlag.FORCE));
     }
     }
 
 
     @Override
     @Override
@@ -858,7 +859,7 @@ public class TestRetryCacheWithHA {
     @Override
     @Override
     void prepare() throws Exception {
     void prepare() throws Exception {
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
-      id = dfs.addCacheDirective(directive);
+      id = dfs.addCacheDirective(directive, EnumSet.of(CacheFlag.FORCE));
     }
     }
 
 
     @Override
     @Override
@@ -936,19 +937,19 @@ public class TestRetryCacheWithHA {
 
 
     @Override
     @Override
     void prepare() throws Exception {
     void prepare() throws Exception {
-      client.addCachePool(new CachePoolInfo(pool).setWeight(10));
+      client.addCachePool(new CachePoolInfo(pool).setLimit(10l));
     }
     }
 
 
     @Override
     @Override
     void invoke() throws Exception {
     void invoke() throws Exception {
-      client.modifyCachePool(new CachePoolInfo(pool).setWeight(99));
+      client.modifyCachePool(new CachePoolInfo(pool).setLimit(99l));
     }
     }
 
 
     @Override
     @Override
     boolean checkNamenodeBeforeReturn() throws Exception {
     boolean checkNamenodeBeforeReturn() throws Exception {
       for (int i = 0; i < CHECKTIMES; i++) {
       for (int i = 0; i < CHECKTIMES; i++) {
         RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
         RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
-        if (iter.hasNext() && iter.next().getInfo().getWeight() == 99) {
+        if (iter.hasNext() && (long)iter.next().getInfo().getLimit() == 99) {
           return true;
           return true;
         }
         }
         Thread.sleep(1000);
         Thread.sleep(1000);
@@ -1216,7 +1217,7 @@ public class TestRetryCacheWithHA {
       CacheDirectiveInfo directiveInfo =
       CacheDirectiveInfo directiveInfo =
         new CacheDirectiveInfo.Builder().setPool(poolName).setPath(path).build();
         new CacheDirectiveInfo.Builder().setPool(poolName).setPath(path).build();
       dfs.addCachePool(new CachePoolInfo(poolName));
       dfs.addCachePool(new CachePoolInfo(poolName));
-      dfs.addCacheDirective(directiveInfo);
+      dfs.addCacheDirective(directiveInfo, EnumSet.of(CacheFlag.FORCE));
       poolNames.add(poolName);
       poolNames.add(poolName);
     }
     }
     listCacheDirectives(poolNames, 0);
     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>
       <TXID>2</TXID>
       <DELEGATION_KEY>
       <DELEGATION_KEY>
         <KEY_ID>1</KEY_ID>
         <KEY_ID>1</KEY_ID>
-        <EXPIRY_DATE>1386695013416</EXPIRY_DATE>
-        <KEY>360a10c6ecac725e</KEY>
+        <EXPIRY_DATE>1387701670577</EXPIRY_DATE>
+        <KEY>7bb5467995769b59</KEY>
       </DELEGATION_KEY>
       </DELEGATION_KEY>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -24,8 +24,8 @@
       <TXID>3</TXID>
       <TXID>3</TXID>
       <DELEGATION_KEY>
       <DELEGATION_KEY>
         <KEY_ID>2</KEY_ID>
         <KEY_ID>2</KEY_ID>
-        <EXPIRY_DATE>1386695013425</EXPIRY_DATE>
-        <KEY>9b110c0b83225f7d</KEY>
+        <EXPIRY_DATE>1387701670580</EXPIRY_DATE>
+        <KEY>a5a3a2755e36827b</KEY>
       </DELEGATION_KEY>
       </DELEGATION_KEY>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -37,17 +37,17 @@
       <INODEID>16386</INODEID>
       <INODEID>16386</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814612</MTIME>
-      <ATIME>1386003814612</ATIME>
+      <MTIME>1387010471220</MTIME>
+      <ATIME>1387010471220</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <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>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </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>
       <RPC_CALLID>7</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -59,13 +59,13 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814665</MTIME>
-      <ATIME>1386003814612</ATIME>
+      <MTIME>1387010471276</MTIME>
+      <ATIME>1387010471220</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -78,8 +78,8 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <SRC>/file_create_u\0001;F431</SRC>
       <SRC>/file_create_u\0001;F431</SRC>
       <DST>/file_moved</DST>
       <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>
       <RPC_CALLID>9</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -89,8 +89,8 @@
       <TXID>7</TXID>
       <TXID>7</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
       <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>
       <RPC_CALLID>10</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -101,9 +101,9 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>16387</INODEID>
       <INODEID>16387</INODEID>
       <PATH>/directory_mkdir</PATH>
       <PATH>/directory_mkdir</PATH>
-      <TIMESTAMP>1386003814686</TIMESTAMP>
+      <TIMESTAMP>1387010471312</TIMESTAMP>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>493</MODE>
         <MODE>493</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -136,7 +136,7 @@
       <TXID>12</TXID>
       <TXID>12</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
       <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>
       <RPC_CALLID>15</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -147,7 +147,7 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
       <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>
       <RPC_CALLID>16</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -157,7 +157,7 @@
       <TXID>14</TXID>
       <TXID>14</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
       <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>
       <RPC_CALLID>17</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -169,17 +169,17 @@
       <INODEID>16388</INODEID>
       <INODEID>16388</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814712</MTIME>
-      <ATIME>1386003814712</ATIME>
+      <MTIME>1387010471373</MTIME>
+      <ATIME>1387010471373</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <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>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </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>
       <RPC_CALLID>18</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -191,13 +191,13 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814714</MTIME>
-      <ATIME>1386003814712</ATIME>
+      <MTIME>1387010471380</MTIME>
+      <ATIME>1387010471373</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -253,9 +253,9 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <SRC>/file_create_u\0001;F431</SRC>
       <SRC>/file_create_u\0001;F431</SRC>
       <DST>/file_moved</DST>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1386003814732</TIMESTAMP>
+      <TIMESTAMP>1387010471428</TIMESTAMP>
       <OPTIONS>NONE</OPTIONS>
       <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>
       <RPC_CALLID>25</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -267,17 +267,17 @@
       <INODEID>16389</INODEID>
       <INODEID>16389</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814737</MTIME>
-      <ATIME>1386003814737</ATIME>
+      <MTIME>1387010471438</MTIME>
+      <ATIME>1387010471438</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <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>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </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>
       <RPC_CALLID>27</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -388,8 +388,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814889</MTIME>
-      <ATIME>1386003814737</ATIME>
+      <MTIME>1387010471540</MTIME>
+      <ATIME>1387010471438</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -409,7 +409,7 @@
         <GENSTAMP>1003</GENSTAMP>
         <GENSTAMP>1003</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -423,17 +423,17 @@
       <INODEID>16390</INODEID>
       <INODEID>16390</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814891</MTIME>
-      <ATIME>1386003814891</ATIME>
+      <MTIME>1387010471547</MTIME>
+      <ATIME>1387010471547</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <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>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </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>
       <RPC_CALLID>40</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -544,8 +544,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814914</MTIME>
-      <ATIME>1386003814891</ATIME>
+      <MTIME>1387010471588</MTIME>
+      <ATIME>1387010471547</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -565,7 +565,7 @@
         <GENSTAMP>1006</GENSTAMP>
         <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -579,17 +579,17 @@
       <INODEID>16391</INODEID>
       <INODEID>16391</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814916</MTIME>
-      <ATIME>1386003814916</ATIME>
+      <MTIME>1387010471595</MTIME>
+      <ATIME>1387010471595</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <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>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </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>
       <RPC_CALLID>52</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -700,8 +700,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814938</MTIME>
-      <ATIME>1386003814916</ATIME>
+      <MTIME>1387010471651</MTIME>
+      <ATIME>1387010471595</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -721,7 +721,7 @@
         <GENSTAMP>1009</GENSTAMP>
         <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -733,12 +733,12 @@
       <TXID>56</TXID>
       <TXID>56</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
       <TRG>/file_concat_target</TRG>
-      <TIMESTAMP>1386003814940</TIMESTAMP>
+      <TIMESTAMP>1387010471663</TIMESTAMP>
       <SOURCES>
       <SOURCES>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE2>/file_concat_1</SOURCE2>
         <SOURCE2>/file_concat_1</SOURCE2>
       </SOURCES>
       </SOURCES>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>63</RPC_CALLID>
       <RPC_CALLID>63</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -750,14 +750,14 @@
       <INODEID>16392</INODEID>
       <INODEID>16392</INODEID>
       <PATH>/file_symlink</PATH>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
       <VALUE>/file_concat_target</VALUE>
-      <MTIME>1386003814956</MTIME>
-      <ATIME>1386003814956</ATIME>
+      <MTIME>1387010471674</MTIME>
+      <ATIME>1387010471674</ATIME>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>511</MODE>
         <MODE>511</MODE>
       </PERMISSION_STATUS>
       </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>
       <RPC_CALLID>64</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -768,14 +768,14 @@
       <DELEGATION_TOKEN_IDENTIFIER>
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
-        <OWNER>aagarwal</OWNER>
+        <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
         <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>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
       </DELEGATION_TOKEN_IDENTIFIER>
-      <EXPIRY_TIME>1386090214961</EXPIRY_TIME>
+      <EXPIRY_TIME>1387096871682</EXPIRY_TIME>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -785,14 +785,14 @@
       <DELEGATION_TOKEN_IDENTIFIER>
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
-        <OWNER>aagarwal</OWNER>
+        <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
         <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>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
       </DELEGATION_TOKEN_IDENTIFIER>
-      <EXPIRY_TIME>1386090215078</EXPIRY_TIME>
+      <EXPIRY_TIME>1387096871717</EXPIRY_TIME>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -802,11 +802,11 @@
       <DELEGATION_TOKEN_IDENTIFIER>
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
-        <OWNER>aagarwal</OWNER>
+        <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
         <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>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
       </DELEGATION_TOKEN_IDENTIFIER>
     </DATA>
     </DATA>
@@ -816,13 +816,11 @@
     <DATA>
     <DATA>
       <TXID>61</TXID>
       <TXID>61</TXID>
       <POOLNAME>poolparty</POOLNAME>
       <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>
       <RPC_CALLID>68</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -834,8 +832,8 @@
       <OWNERNAME>carlton</OWNERNAME>
       <OWNERNAME>carlton</OWNERNAME>
       <GROUPNAME>party</GROUPNAME>
       <GROUPNAME>party</GROUPNAME>
       <MODE>448</MODE>
       <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>
       <RPC_CALLID>69</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -848,7 +846,7 @@
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <POOL>poolparty</POOL>
       <POOL>poolparty</POOL>
       <EXPIRATION>-1</EXPIRATION>
       <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>
       <RPC_CALLID>70</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -858,7 +856,7 @@
       <TXID>64</TXID>
       <TXID>64</TXID>
       <ID>1</ID>
       <ID>1</ID>
       <PATH>/bar2</PATH>
       <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>
       <RPC_CALLID>71</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -867,7 +865,7 @@
     <DATA>
     <DATA>
       <TXID>65</TXID>
       <TXID>65</TXID>
       <ID>1</ID>
       <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>
       <RPC_CALLID>72</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -876,7 +874,7 @@
     <DATA>
     <DATA>
       <TXID>66</TXID>
       <TXID>66</TXID>
       <POOLNAME>poolparty</POOLNAME>
       <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>
       <RPC_CALLID>73</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -888,17 +886,17 @@
       <INODEID>16393</INODEID>
       <INODEID>16393</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003815135</MTIME>
-      <ATIME>1386003815135</ATIME>
+      <MTIME>1387010471802</MTIME>
+      <ATIME>1387010471802</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <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>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </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>
       <RPC_CALLID>74</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -955,7 +953,7 @@
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
     <DATA>
       <TXID>73</TXID>
       <TXID>73</TXID>
-      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-1253204429_1</LEASEHOLDER>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-52011019_1</LEASEHOLDER>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
     </DATA>
     </DATA>
@@ -968,8 +966,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003817462</MTIME>
-      <ATIME>1386003815135</ATIME>
+      <MTIME>1387010474126</MTIME>
+      <ATIME>1387010471802</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -979,7 +977,7 @@
         <GENSTAMP>1011</GENSTAMP>
         <GENSTAMP>1011</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>

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

@@ -80,8 +80,8 @@
     <test> <!--Tested -->
     <test> <!--Tested -->
       <description>Testing modifying a cache pool</description>
       <description>Testing modifying a cache pool</description>
       <test-commands>
       <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>
         <cache-admin-command>-listPools</cache-admin-command>
       </test-commands>
       </test-commands>
       <cleanup-commands>
       <cleanup-commands>
@@ -90,7 +90,7 @@
       <comparators>
       <comparators>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>poolparty  bob    bobgroup  rwxrwxrwx       51</expected-output>
+          <expected-output>poolparty  bob    bobgroup  rwxrwxrwx      51</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -129,11 +129,11 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <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>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <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>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -156,7 +156,7 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <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>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -417,11 +417,11 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <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>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <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>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>

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

@@ -71,6 +71,12 @@ Trunk (Unreleased)
     MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing.
     MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing.
     (Srikanth Sundarrajan via amareshwari)
     (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
   BUG FIXES
 
 
     MAPREDUCE-4272. SortedRanges.Range#compareTo is not spec compliant.
     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
     MAPREDUCE-5623. TestJobCleanup fails because of RejectedExecutionException
     and NPE. (jlowe)
     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
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   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;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
 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.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.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.PolicyProvider;
@@ -84,14 +85,17 @@ public class TaskAttemptListenerImpl extends CompositeService
       .newSetFromMap(new ConcurrentHashMap<WrappedJvmID, Boolean>()); 
       .newSetFromMap(new ConcurrentHashMap<WrappedJvmID, Boolean>()); 
   
   
   private JobTokenSecretManager jobTokenSecretManager = null;
   private JobTokenSecretManager jobTokenSecretManager = null;
+  private AMPreemptionPolicy preemptionPolicy;
   
   
   public TaskAttemptListenerImpl(AppContext context,
   public TaskAttemptListenerImpl(AppContext context,
       JobTokenSecretManager jobTokenSecretManager,
       JobTokenSecretManager jobTokenSecretManager,
-      RMHeartbeatHandler rmHeartbeatHandler) {
+      RMHeartbeatHandler rmHeartbeatHandler,
+      AMPreemptionPolicy preemptionPolicy) {
     super(TaskAttemptListenerImpl.class.getName());
     super(TaskAttemptListenerImpl.class.getName());
     this.context = context;
     this.context = context;
     this.jobTokenSecretManager = jobTokenSecretManager;
     this.jobTokenSecretManager = jobTokenSecretManager;
     this.rmHeartbeatHandler = rmHeartbeatHandler;
     this.rmHeartbeatHandler = rmHeartbeatHandler;
+    this.preemptionPolicy = preemptionPolicy;
   }
   }
 
 
   @Override
   @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.RMContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerRequestor;
 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.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.DefaultSpeculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
@@ -188,8 +190,8 @@ public class MRAppMaster extends CompositeService {
   private ContainerLauncher containerLauncher;
   private ContainerLauncher containerLauncher;
   private EventHandler<CommitterEvent> committerEventHandler;
   private EventHandler<CommitterEvent> committerEventHandler;
   private Speculator speculator;
   private Speculator speculator;
-  private TaskAttemptListener taskAttemptListener;
-  private JobTokenSecretManager jobTokenSecretManager =
+  protected TaskAttemptListener taskAttemptListener;
+  protected JobTokenSecretManager jobTokenSecretManager =
       new JobTokenSecretManager();
       new JobTokenSecretManager();
   private JobId jobId;
   private JobId jobId;
   private boolean newApiCommitter;
   private boolean newApiCommitter;
@@ -197,6 +199,7 @@ public class MRAppMaster extends CompositeService {
   private JobEventDispatcher jobEventDispatcher;
   private JobEventDispatcher jobEventDispatcher;
   private JobHistoryEventHandler jobHistoryEventHandler;
   private JobHistoryEventHandler jobHistoryEventHandler;
   private SpeculatorEventDispatcher speculatorEventDispatcher;
   private SpeculatorEventDispatcher speculatorEventDispatcher;
+  private AMPreemptionPolicy preemptionPolicy;
 
 
   private Job job;
   private Job job;
   private Credentials jobCredentials = new Credentials(); // Filled during init
   private Credentials jobCredentials = new Credentials(); // Filled during init
@@ -383,8 +386,12 @@ public class MRAppMaster extends CompositeService {
       committerEventHandler = createCommitterEventHandler(context, committer);
       committerEventHandler = createCommitterEventHandler(context, committer);
       addIfService(committerEventHandler);
       addIfService(committerEventHandler);
 
 
+      //policy handling preemption requests from RM
+      preemptionPolicy = createPreemptionPolicy(conf);
+      preemptionPolicy.init(context);
+
       //service to handle requests to TaskUmbilicalProtocol
       //service to handle requests to TaskUmbilicalProtocol
-      taskAttemptListener = createTaskAttemptListener(context);
+      taskAttemptListener = createTaskAttemptListener(context, preemptionPolicy);
       addIfService(taskAttemptListener);
       addIfService(taskAttemptListener);
 
 
       //service to log job history events
       //service to log job history events
@@ -475,6 +482,12 @@ public class MRAppMaster extends CompositeService {
     return committer;
     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) {
   protected boolean keepJobFiles(JobConf conf) {
     return (conf.getKeepTaskFilesPattern() != null || conf
     return (conf.getKeepTaskFilesPattern() != null || conf
         .getKeepFailedTaskFiles());
         .getKeepFailedTaskFiles());
@@ -692,10 +705,11 @@ public class MRAppMaster extends CompositeService {
     }
     }
   }
   }
 
 
-  protected TaskAttemptListener createTaskAttemptListener(AppContext context) {
+  protected TaskAttemptListener createTaskAttemptListener(AppContext context,
+      AMPreemptionPolicy preemptionPolicy) {
     TaskAttemptListener lis =
     TaskAttemptListener lis =
         new TaskAttemptListenerImpl(context, jobTokenSecretManager,
         new TaskAttemptListenerImpl(context, jobTokenSecretManager,
-            getRMHeartbeatHandler());
+            getRMHeartbeatHandler(), preemptionPolicy);
     return lis;
     return lis;
   }
   }
 
 
@@ -805,7 +819,7 @@ public class MRAppMaster extends CompositeService {
             , containerID);
             , containerID);
       } else {
       } else {
         this.containerAllocator = new RMContainerAllocator(
         this.containerAllocator = new RMContainerAllocator(
-            this.clientService, this.context);
+            this.clientService, this.context, preemptionPolicy);
       }
       }
       ((Service)this.containerAllocator).init(getConfig());
       ((Service)this.containerAllocator).init(getConfig());
       ((Service)this.containerAllocator).start();
       ((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.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 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.job.event.TaskAttemptKillEvent;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 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.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 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.api.records.Priority;
 import org.apache.hadoop.yarn.client.api.NMTokenCache;
 import org.apache.hadoop.yarn.client.api.NMTokenCache;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -147,13 +149,17 @@ public class RMContainerAllocator extends RMContainerRequestor
   private long retryInterval;
   private long retryInterval;
   private long retrystartTime;
   private long retrystartTime;
 
 
+  private final AMPreemptionPolicy preemptionPolicy;
+
   BlockingQueue<ContainerAllocatorEvent> eventQueue
   BlockingQueue<ContainerAllocatorEvent> eventQueue
     = new LinkedBlockingQueue<ContainerAllocatorEvent>();
     = new LinkedBlockingQueue<ContainerAllocatorEvent>();
 
 
   private ScheduleStats scheduleStats = new ScheduleStats();
   private ScheduleStats scheduleStats = new ScheduleStats();
 
 
-  public RMContainerAllocator(ClientService clientService, AppContext context) {
+  public RMContainerAllocator(ClientService clientService, AppContext context,
+      AMPreemptionPolicy preemptionPolicy) {
     super(clientService, context);
     super(clientService, context);
+    this.preemptionPolicy = preemptionPolicy;
     this.stopped = new AtomicBoolean(false);
     this.stopped = new AtomicBoolean(false);
   }
   }
 
 
@@ -361,11 +367,15 @@ public class RMContainerAllocator extends RMContainerRequestor
         LOG.error("Could not deallocate container for task attemptId " + 
         LOG.error("Could not deallocate container for task attemptId " + 
             aId);
             aId);
       }
       }
+      preemptionPolicy.handleCompletedContainer(event.getAttemptID());
     } else if (
     } else if (
         event.getType() == ContainerAllocator.EventType.CONTAINER_FAILED) {
         event.getType() == ContainerAllocator.EventType.CONTAINER_FAILED) {
       ContainerFailedEvent fEv = (ContainerFailedEvent) event;
       ContainerFailedEvent fEv = (ContainerFailedEvent) event;
       String host = getHost(fEv.getContMgrAddress());
       String host = getHost(fEv.getContMgrAddress());
       containerFailedOnHost(host);
       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();
         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 premeptionLimit = Math.max(mapResourceReqt, 
             (int) (maxReducePreemptionLimit * memLimit));
             (int) (maxReducePreemptionLimit * memLimit));
         
         
@@ -409,7 +419,7 @@ public class RMContainerAllocator extends RMContainerRequestor
         int toPreempt = (int) Math.ceil((float) preemptMem/reduceResourceReqt);
         int toPreempt = (int) Math.ceil((float) preemptMem/reduceResourceReqt);
         toPreempt = Math.min(toPreempt, assignedRequests.reduces.size());
         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);
         assignedRequests.preemptReduce(toPreempt);
       }
       }
     }
     }
@@ -595,6 +605,14 @@ public class RMContainerAllocator extends RMContainerRequestor
     }
     }
     
     
     List<ContainerStatus> finishedContainers = response.getCompletedContainersStatuses();
     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) {
     if (newContainers.size() + finishedContainers.size() > 0 || headRoom != newHeadRoom) {
       //something changed
       //something changed
       recalculateReduceSchedule = true;
       recalculateReduceSchedule = true;
@@ -630,7 +648,9 @@ public class RMContainerAllocator extends RMContainerRequestor
         String diagnostics = StringInterner.weakIntern(cont.getDiagnostics());
         String diagnostics = StringInterner.weakIntern(cont.getDiagnostics());
         eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptID,
         eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptID,
             diagnostics));
             diagnostics));
-      }      
+
+        preemptionPolicy.handleCompletedContainer(attemptID);
+      }
     }
     }
     return newContainers;
     return newContainers;
   }
   }
@@ -1232,4 +1252,27 @@ public class RMContainerAllocator extends RMContainerRequestor
         " RackLocal:" + rackLocalAssigned);
         " 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,
         JobTokenSecretManager jobTokenSecretManager,
         RMHeartbeatHandler rmHeartbeatHandler,
         RMHeartbeatHandler rmHeartbeatHandler,
         TaskHeartbeatHandler hbHandler) {
         TaskHeartbeatHandler hbHandler) {
-      super(context, jobTokenSecretManager, rmHeartbeatHandler);
+      super(context, jobTokenSecretManager, rmHeartbeatHandler, null);
       this.taskHeartbeatHandler = hbHandler;
       this.taskHeartbeatHandler = hbHandler;
     }
     }
     
     
@@ -191,7 +191,7 @@ public class TestTaskAttemptListenerImpl {
         mock(RMHeartbeatHandler.class);
         mock(RMHeartbeatHandler.class);
     final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
     final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
     TaskAttemptListenerImpl listener =
     TaskAttemptListenerImpl listener =
-        new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler) {
+        new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler, null) {
       @Override
       @Override
       protected void registerHeartbeatHandler(Configuration conf) {
       protected void registerHeartbeatHandler(Configuration conf) {
         taskHeartbeatHandler = hbHandler;
         taskHeartbeatHandler = hbHandler;
@@ -245,7 +245,7 @@ public class TestTaskAttemptListenerImpl {
         mock(RMHeartbeatHandler.class);
         mock(RMHeartbeatHandler.class);
     final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
     final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
     TaskAttemptListenerImpl listener =
     TaskAttemptListenerImpl listener =
-        new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler) {
+        new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler, null) {
       @Override
       @Override
       protected void registerHeartbeatHandler(Configuration conf) {
       protected void registerHeartbeatHandler(Configuration conf) {
         taskHeartbeatHandler = hbHandler;
         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.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 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.RMHeartbeatHandler;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -467,7 +468,8 @@ public class MRApp extends MRAppMaster {
   }
   }
 
 
   @Override
   @Override
-  protected TaskAttemptListener createTaskAttemptListener(AppContext context) {
+  protected TaskAttemptListener createTaskAttemptListener(
+      AppContext context, AMPreemptionPolicy policy) {
     return new TaskAttemptListener(){
     return new TaskAttemptListener(){
       @Override
       @Override
       public InetSocketAddress getAddress() {
       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.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 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.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.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@@ -61,6 +63,8 @@ public class MRAppBenchmark {
 
 
   /**
   /**
    * Runs memory and time benchmark with Mock MRApp.
    * 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 {
   public void run(MRApp app) throws Exception {
     Logger rootLogger = LogManager.getRootLogger();
     Logger rootLogger = LogManager.getRootLogger();
@@ -133,6 +137,7 @@ public class MRAppBenchmark {
       protected void serviceStart() throws Exception {
       protected void serviceStart() throws Exception {
         thread = new Thread(new Runnable() {
         thread = new Thread(new Runnable() {
           @Override
           @Override
+          @SuppressWarnings("unchecked")
           public void run() {
           public void run() {
             ContainerAllocatorEvent event = null;
             ContainerAllocatorEvent event = null;
             while (!Thread.currentThread().isInterrupted()) {
             while (!Thread.currentThread().isInterrupted()) {
@@ -192,7 +197,9 @@ public class MRAppBenchmark {
       @Override
       @Override
       protected ContainerAllocator createContainerAllocator(
       protected ContainerAllocator createContainerAllocator(
           ClientService clientService, AppContext context) {
           ClientService clientService, AppContext context) {
-        return new RMContainerAllocator(clientService, context) {
+
+        AMPreemptionPolicy policy = new NoopAMPreemptionPolicy();
+        return new RMContainerAllocator(clientService, context, policy) {
           @Override
           @Override
           protected ApplicationMasterProtocol createSchedulerProxy() {
           protected ApplicationMasterProtocol createSchedulerProxy() {
             return new ApplicationMasterProtocol() {
             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.ContainerLauncher;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 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.launcher.ContainerLauncherImpl;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -247,13 +248,14 @@ public class TestFail {
       super(maps, reduces, false, "TimeOutTaskMRApp", true);
       super(maps, reduces, false, "TimeOutTaskMRApp", true);
     }
     }
     @Override
     @Override
-    protected TaskAttemptListener createTaskAttemptListener(AppContext context) {
+    protected TaskAttemptListener createTaskAttemptListener(
+        AppContext context, AMPreemptionPolicy policy) {
       //This will create the TaskAttemptListener with TaskHeartbeatHandler
       //This will create the TaskAttemptListener with TaskHeartbeatHandler
       //RPC servers are not started
       //RPC servers are not started
       //task time out is reduced
       //task time out is reduced
       //when attempt times out, heartbeat handler will send the lost event
       //when attempt times out, heartbeat handler will send the lost event
       //leading to Attempt failure
       //leading to Attempt failure
-      return new TaskAttemptListenerImpl(getContext(), null, null) {
+      return new TaskAttemptListenerImpl(getContext(), null, null, policy) {
         @Override
         @Override
         public void startRpcServer(){};
         public void startRpcServer(){};
         @Override
         @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;
 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.anyFloat;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Matchers.isA;
@@ -1428,14 +1430,15 @@ public class TestRMContainerAllocator {
     // Use this constructor when using a real job.
     // Use this constructor when using a real job.
     MyContainerAllocator(MyResourceManager rm,
     MyContainerAllocator(MyResourceManager rm,
         ApplicationAttemptId appAttemptId, AppContext context) {
         ApplicationAttemptId appAttemptId, AppContext context) {
-      super(createMockClientService(), context);
+      super(createMockClientService(), context, new NoopAMPreemptionPolicy());
       this.rm = rm;
       this.rm = rm;
     }
     }
 
 
     // Use this constructor when you are using a mocked job.
     // Use this constructor when you are using a mocked job.
     public MyContainerAllocator(MyResourceManager rm, Configuration conf,
     public MyContainerAllocator(MyResourceManager rm, Configuration conf,
         ApplicationAttemptId appAttemptId, Job job) {
         ApplicationAttemptId appAttemptId, Job job) {
-      super(createMockClientService(), createAppContext(appAttemptId, job));
+      super(createMockClientService(), createAppContext(appAttemptId, job),
+          new NoopAMPreemptionPolicy());
       this.rm = rm;
       this.rm = rm;
       super.init(conf);
       super.init(conf);
       super.start();
       super.start();
@@ -1444,7 +1447,8 @@ public class TestRMContainerAllocator {
     public MyContainerAllocator(MyResourceManager rm, Configuration conf,
     public MyContainerAllocator(MyResourceManager rm, Configuration conf,
         ApplicationAttemptId appAttemptId, Job job, Clock clock) {
         ApplicationAttemptId appAttemptId, Job job, Clock clock) {
       super(createMockClientService(),
       super(createMockClientService(),
-          createAppContext(appAttemptId, job, clock));
+          createAppContext(appAttemptId, job, clock),
+          new NoopAMPreemptionPolicy());
       this.rm = rm;
       this.rm = rm;
       super.init(conf);
       super.init(conf);
       super.start();
       super.start();
@@ -1671,7 +1675,8 @@ public class TestRMContainerAllocator {
         ApplicationId.newInstance(1, 1));
         ApplicationId.newInstance(1, 1));
 
 
     RMContainerAllocator allocator = new RMContainerAllocator(
     RMContainerAllocator allocator = new RMContainerAllocator(
-        mock(ClientService.class), appContext) {
+        mock(ClientService.class), appContext,
+        new NoopAMPreemptionPolicy()) {
           @Override
           @Override
           protected void register() {
           protected void register() {
           }
           }
@@ -1721,7 +1726,8 @@ public class TestRMContainerAllocator {
   @Test
   @Test
   public void testCompletedContainerEvent() {
   public void testCompletedContainerEvent() {
     RMContainerAllocator allocator = new RMContainerAllocator(
     RMContainerAllocator allocator = new RMContainerAllocator(
-        mock(ClientService.class), mock(AppContext.class));
+        mock(ClientService.class), mock(AppContext.class),
+        new NoopAMPreemptionPolicy());
     
     
     TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(
     TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(
         MRBuilderUtils.newTaskId(
         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,
   TOTAL_LAUNCHED_UBERTASKS,
   NUM_UBER_SUBMAPS,
   NUM_UBER_SUBMAPS,
   NUM_UBER_SUBREDUCES,
   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 = 
   public static final String MR_AM_JOB_REDUCE_PREEMPTION_LIMIT = 
     MR_AM_PREFIX  + "job.reduce.preemption.limit";
     MR_AM_PREFIX  + "job.reduce.preemption.limit";
   public static final float DEFAULT_MR_AM_JOB_REDUCE_PREEMPTION_LIMIT = 0.5f;
   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. **/
   /** AM ACL disabled. **/
   public static final String JOB_AM_ACCESS_DISABLED = 
   public static final String JOB_AM_ACCESS_DISABLED = 
     "mapreduce.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 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)
 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_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)
 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;
     PrintStream stdps = System.out;
     try {
     try {
       System.setOut(new PrintStream(outContent));
       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();
       viewer.print();
 
 
       for (TaskInfo taskInfo : allTasks.values()) {
       for (TaskInfo taskInfo : allTasks.values()) {
@@ -397,29 +400,27 @@ public class TestJobHistoryParsing {
       // make sure all events are flushed
       // make sure all events are flushed
       app.waitForState(Service.STATE.STOPPED);
       app.waitForState(Service.STATE.STOPPED);
 
 
-      String jobhistoryDir = JobHistoryUtils
-          .getHistoryIntermediateDoneDirForUser(conf);
       JobHistory jobHistory = new JobHistory();
       JobHistory jobHistory = new JobHistory();
       jobHistory.init(conf);
       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();
       Exception parseException = parser.getParseException();
       Assert.assertNull("Caught an expected exception " + parseException,
       Assert.assertNull("Caught an expected exception " + parseException,
           parseException);
           parseException);
@@ -464,29 +465,28 @@ public class TestJobHistoryParsing {
       // make sure all events are flushed
       // make sure all events are flushed
       app.waitForState(Service.STATE.STOPPED);
       app.waitForState(Service.STATE.STOPPED);
 
 
-      String jobhistoryDir = JobHistoryUtils
-          .getHistoryIntermediateDoneDirForUser(conf);
       JobHistory jobHistory = new JobHistory();
       JobHistory jobHistory = new JobHistory();
       jobHistory.init(conf);
       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();
       Exception parseException = parser.getParseException();
       Assert.assertNull("Caught an expected exception " + parseException,
       Assert.assertNull("Caught an expected exception " + parseException,
           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
     @Override
     public KillApplicationResponse forceKillApplication(
     public KillApplicationResponse forceKillApplication(
         KillApplicationRequest request) throws IOException {
         KillApplicationRequest request) throws IOException {
-      return recordFactory.newRecordInstance(KillApplicationResponse.class);
+      return KillApplicationResponse.newInstance(true);
     }
     }
 
 
     @Override
     @Override

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