Browse Source

Merge HDFS-4949 changes from trunk to branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1560522 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 11 years ago
parent
commit
c4194399d3
100 changed files with 11426 additions and 222 deletions
  1. 2 2
      dev-support/test-patch.sh
  2. 118 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchedRemoteIterator.java
  3. 66 35
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BlockLocation.java
  4. 32 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidRequestException.java
  5. 7 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
  6. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java
  7. 1 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
  8. 165 11
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
  9. 8 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  10. 373 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IntrusiveCollection.java
  11. 25 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java
  12. 35 14
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java
  13. 79 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
  14. 87 2
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
  15. 108 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestBlockLocation.java
  16. 61 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
  17. 110 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java
  18. 208 1
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  19. 16 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  20. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  21. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/CacheFlag.java
  22. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
  23. 74 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  24. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  25. 79 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  26. 159 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  27. 2 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ClientMmap.java
  28. 104 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  29. 268 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java
  30. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
  31. 358 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
  32. 56 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
  33. 169 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
  34. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java
  35. 229 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
  36. 53 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
  37. 115 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
  38. 90 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  39. 77 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  40. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  41. 56 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  42. 135 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  43. 179 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  44. 35 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  45. 26 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
  46. 304 30
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  47. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  48. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  49. 774 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  50. 113 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  51. 99 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  52. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
  53. 22 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
  54. 34 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  55. 43 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  56. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  57. 9 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  58. 20 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
  59. 29 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  60. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  61. 506 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java
  62. 105 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  63. 34 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  64. 174 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MappableBlock.java
  65. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
  66. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java
  67. 1073 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  68. 328 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
  69. 251 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachedBlock.java
  70. 16 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  71. 63 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  72. 63 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  73. 414 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  74. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
  75. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  76. 224 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  77. 296 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  78. 29 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  79. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  80. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
  81. 70 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  82. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  83. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java
  84. 50 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockIdCommand.java
  85. 24 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
  86. 1059 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
  87. 284 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/TableListing.java
  88. 39 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  89. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java
  90. 16 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java
  91. 42 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  92. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
  93. 132 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  94. 44 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
  95. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  96. 96 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  97. 301 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/CentralizedCacheManagement.apt.vm
  98. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/caching.png
  99. 141 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCacheAdminCLI.java
  100. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandCacheAdmin.java

+ 2 - 2
dev-support/test-patch.sh

@@ -395,9 +395,9 @@ checkJavadocWarnings () {
   echo ""
   echo ""
   echo "There appear to be $javadocWarnings javadoc warnings generated by the patched build."
   echo "There appear to be $javadocWarnings javadoc warnings generated by the patched build."
 
 
-  #There are 11 warnings that are caused by things that are caused by using sun internal APIs.
+  #There are 12 warnings that are caused by things that are caused by using sun internal APIs.
   #There are 2 warnings that are caused by the Apache DS Dn class used in MiniKdc.
   #There are 2 warnings that are caused by the Apache DS Dn class used in MiniKdc.
-  OK_JAVADOC_WARNINGS=13;
+  OK_JAVADOC_WARNINGS=14;
   ### if current warnings greater than OK_JAVADOC_WARNINGS
   ### if current warnings greater than OK_JAVADOC_WARNINGS
   if [[ $javadocWarnings -gt $OK_JAVADOC_WARNINGS ]] ; then
   if [[ $javadocWarnings -gt $OK_JAVADOC_WARNINGS ]] ; then
     JIRA_COMMENT="$JIRA_COMMENT
     JIRA_COMMENT="$JIRA_COMMENT

+ 118 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchedRemoteIterator.java

@@ -0,0 +1,118 @@
+/**
+ * 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 java.io.IOException;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+/**
+ * A RemoteIterator that fetches elements in batches.
+ */
+public abstract class BatchedRemoteIterator<K, E> implements RemoteIterator<E> {
+  public interface BatchedEntries<E> {
+    public E get(int i);
+    public int size();
+    public boolean hasMore();
+  }
+
+  public static class BatchedListEntries<E> implements BatchedEntries<E> {
+    private final List<E> entries;
+    private final boolean hasMore;
+
+    public BatchedListEntries(List<E> entries, boolean hasMore) {
+      this.entries = entries;
+      this.hasMore = hasMore;
+    }
+
+    public E get(int i) {
+      return entries.get(i);
+    }
+
+    public int size() {
+      return entries.size();
+    }
+
+    public boolean hasMore() {
+      return hasMore;
+    }
+  }
+
+  private K prevKey;
+  private BatchedEntries<E> entries;
+  private int idx;
+
+  public BatchedRemoteIterator(K prevKey) {
+    this.prevKey = prevKey;
+    this.entries = null;
+    this.idx = -1;
+  }
+
+  /**
+   * Perform the actual remote request.
+   * 
+   * @param prevKey The key to send.
+   * @return A list of replies.
+   */
+  public abstract BatchedEntries<E> makeRequest(K prevKey) throws IOException;
+
+  private void makeRequest() throws IOException {
+    idx = 0;
+    entries = null;
+    entries = makeRequest(prevKey);
+    if (entries.size() == 0) {
+      entries = null;
+    }
+  }
+
+  private void makeRequestIfNeeded() throws IOException {
+    if (idx == -1) {
+      makeRequest();
+    } else if ((entries != null) && (idx >= entries.size())) {
+      if (!entries.hasMore()) {
+        // Last time, we got fewer entries than requested.
+        // So we should be at the end.
+        entries = null;
+      } else {
+        makeRequest();
+      }
+    }
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    makeRequestIfNeeded();
+    return (entries != null);
+  }
+
+  /**
+   * Return the next list key associated with an element.
+   */
+  public abstract K elementToPrevKey(E element);
+
+  @Override
+  public E next() throws IOException {
+    makeRequestIfNeeded();
+    if (entries == null) {
+      throw new NoSuchElementException();
+    }
+    E entry = entries.get(idx++);
+    prevKey = elementToPrevKey(entry);
+    return entry;
+  }
+}

+ 66 - 35
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BlockLocation.java

@@ -31,17 +31,33 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Stable
 @InterfaceStability.Stable
 public class BlockLocation {
 public class BlockLocation {
   private String[] hosts; // Datanode hostnames
   private String[] hosts; // Datanode hostnames
+  private String[] cachedHosts; // Datanode hostnames with a cached replica
   private String[] names; // Datanode IP:xferPort for accessing the block
   private String[] names; // Datanode IP:xferPort for accessing the block
   private String[] topologyPaths; // Full path name in network topology
   private String[] topologyPaths; // Full path name in network topology
   private long offset;  // Offset of the block in the file
   private long offset;  // Offset of the block in the file
   private long length;
   private long length;
   private boolean corrupt;
   private boolean corrupt;
 
 
+  private static final String[] EMPTY_STR_ARRAY = new String[0];
+
   /**
   /**
    * Default Constructor
    * Default Constructor
    */
    */
   public BlockLocation() {
   public BlockLocation() {
-    this(new String[0], new String[0],  0L, 0L);
+    this(EMPTY_STR_ARRAY, EMPTY_STR_ARRAY, 0L, 0L);
+  }
+
+  /**
+   * Copy constructor
+   */
+  public BlockLocation(BlockLocation that) {
+    this.hosts = that.hosts;
+    this.cachedHosts = that.cachedHosts;
+    this.names = that.names;
+    this.topologyPaths = that.topologyPaths;
+    this.offset = that.offset;
+    this.length = that.length;
+    this.corrupt = that.corrupt;
   }
   }
 
 
   /**
   /**
@@ -57,20 +73,7 @@ public class BlockLocation {
    */
    */
   public BlockLocation(String[] names, String[] hosts, long offset, 
   public BlockLocation(String[] names, String[] hosts, long offset, 
                        long length, boolean corrupt) {
                        long length, boolean corrupt) {
-    if (names == null) {
-      this.names = new String[0];
-    } else {
-      this.names = names;
-    }
-    if (hosts == null) {
-      this.hosts = new String[0];
-    } else {
-      this.hosts = hosts;
-    }
-    this.offset = offset;
-    this.length = length;
-    this.topologyPaths = new String[0];
-    this.corrupt = corrupt;
+    this(names, hosts, null, offset, length, corrupt);
   }
   }
 
 
   /**
   /**
@@ -87,34 +90,55 @@ public class BlockLocation {
    */
    */
   public BlockLocation(String[] names, String[] hosts, String[] topologyPaths,
   public BlockLocation(String[] names, String[] hosts, String[] topologyPaths,
                        long offset, long length, boolean corrupt) {
                        long offset, long length, boolean corrupt) {
-    this(names, hosts, offset, length, corrupt);
+    this(names, hosts, null, topologyPaths, offset, length, corrupt);
+  }
+
+  public BlockLocation(String[] names, String[] hosts, String[] cachedHosts,
+      String[] topologyPaths, long offset, long length, boolean corrupt) {
+    if (names == null) {
+      this.names = EMPTY_STR_ARRAY;
+    } else {
+      this.names = names;
+    }
+    if (hosts == null) {
+      this.hosts = EMPTY_STR_ARRAY;
+    } else {
+      this.hosts = hosts;
+    }
+    if (cachedHosts == null) {
+      this.cachedHosts = EMPTY_STR_ARRAY;
+    } else {
+      this.cachedHosts = cachedHosts;
+    }
     if (topologyPaths == null) {
     if (topologyPaths == null) {
-      this.topologyPaths = new String[0];
+      this.topologyPaths = EMPTY_STR_ARRAY;
     } else {
     } else {
       this.topologyPaths = topologyPaths;
       this.topologyPaths = topologyPaths;
     }
     }
+    this.offset = offset;
+    this.length = length;
+    this.corrupt = corrupt;
   }
   }
 
 
   /**
   /**
    * Get the list of hosts (hostname) hosting this block
    * Get the list of hosts (hostname) hosting this block
    */
    */
   public String[] getHosts() throws IOException {
   public String[] getHosts() throws IOException {
-    if (hosts == null || hosts.length == 0) {
-      return new String[0];
-    } else {
-      return hosts;
-    }
+    return hosts;
+  }
+
+  /**
+   * Get the list of hosts (hostname) hosting a cached replica of the block
+   */
+  public String[] getCachedHosts() {
+   return cachedHosts;
   }
   }
 
 
   /**
   /**
    * Get the list of names (IP:xferPort) hosting this block
    * Get the list of names (IP:xferPort) hosting this block
    */
    */
   public String[] getNames() throws IOException {
   public String[] getNames() throws IOException {
-    if (names == null || names.length == 0) {
-      return new String[0];
-    } else {
-      return names;
-    }
+    return names;
   }
   }
 
 
   /**
   /**
@@ -122,11 +146,7 @@ public class BlockLocation {
    * The last component of the path is the "name" (IP:xferPort).
    * The last component of the path is the "name" (IP:xferPort).
    */
    */
   public String[] getTopologyPaths() throws IOException {
   public String[] getTopologyPaths() throws IOException {
-    if (topologyPaths == null || topologyPaths.length == 0) {
-      return new String[0];
-    } else {
-      return topologyPaths;
-    }
+    return topologyPaths;
   }
   }
   
   
   /**
   /**
@@ -176,18 +196,29 @@ public class BlockLocation {
    */
    */
   public void setHosts(String[] hosts) throws IOException {
   public void setHosts(String[] hosts) throws IOException {
     if (hosts == null) {
     if (hosts == null) {
-      this.hosts = new String[0];
+      this.hosts = EMPTY_STR_ARRAY;
     } else {
     } else {
       this.hosts = hosts;
       this.hosts = hosts;
     }
     }
   }
   }
 
 
+  /**
+   * Set the hosts hosting a cached replica of this block
+   */
+  public void setCachedHosts(String[] cachedHosts) {
+    if (cachedHosts == null) {
+      this.cachedHosts = EMPTY_STR_ARRAY;
+    } else {
+      this.cachedHosts = cachedHosts;
+    }
+  }
+
   /**
   /**
    * Set the names (host:port) hosting this block
    * Set the names (host:port) hosting this block
    */
    */
   public void setNames(String[] names) throws IOException {
   public void setNames(String[] names) throws IOException {
     if (names == null) {
     if (names == null) {
-      this.names = new String[0];
+      this.names = EMPTY_STR_ARRAY;
     } else {
     } else {
       this.names = names;
       this.names = names;
     }
     }
@@ -198,7 +229,7 @@ public class BlockLocation {
    */
    */
   public void setTopologyPaths(String[] topologyPaths) throws IOException {
   public void setTopologyPaths(String[] topologyPaths) throws IOException {
     if (topologyPaths == null) {
     if (topologyPaths == null) {
-      this.topologyPaths = new String[0];
+      this.topologyPaths = EMPTY_STR_ARRAY;
     } else {
     } else {
       this.topologyPaths = topologyPaths;
       this.topologyPaths = topologyPaths;
     }
     }

+ 32 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidRequestException.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.fs;
+
+import java.io.IOException;
+
+/**
+ * Thrown when the user makes a malformed request, for example missing required
+ * parameters or parameters that are not valid.
+ */
+public class InvalidRequestException extends IOException {
+  static final long serialVersionUID = 0L;
+
+  public InvalidRequestException(String str) {
+    super(str);
+  }
+}

+ 7 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -304,6 +304,13 @@ public class FsPermission implements Writable {
     return new FsPermission((short)00666);
     return new FsPermission((short)00666);
   }
   }
 
 
+  /**
+   * Get the default permission for cache pools.
+   */
+  public static FsPermission getCachePoolDefault() {
+    return new FsPermission((short)00755);
+  }
+
   /**
   /**
    * Create a FsPermission from a Unix symbolic permission string
    * Create a FsPermission from a Unix symbolic permission string
    * @param unixSymbolicPermission e.g. "-rw-rw-rw-"
    * @param unixSymbolicPermission e.g. "-rw-rw-rw-"

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java

@@ -203,8 +203,8 @@ public class ReadaheadPool {
       // It's also possible that we'll end up requesting readahead on some
       // It's also possible that we'll end up requesting readahead on some
       // other FD, which may be wasted work, but won't cause a problem.
       // other FD, which may be wasted work, but won't cause a problem.
       try {
       try {
-        NativeIO.POSIX.posixFadviseIfPossible(identifier, fd, off, len,
-            NativeIO.POSIX.POSIX_FADV_WILLNEED);
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
+            fd, off, len, NativeIO.POSIX.POSIX_FADV_WILLNEED);
       } catch (IOException ioe) {
       } catch (IOException ioe) {
         if (canceled) {
         if (canceled) {
           // no big deal - the reader canceled the request and closed
           // no big deal - the reader canceled the request and closed

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

@@ -454,10 +454,7 @@ public class Text extends BinaryComparable
   /** Read a UTF8 encoded string from in
   /** Read a UTF8 encoded string from in
    */
    */
   public static String readString(DataInput in) throws IOException {
   public static String readString(DataInput in) throws IOException {
-    int length = WritableUtils.readVInt(in);
-    byte [] bytes = new byte[length];
-    in.readFully(bytes, 0, length);
-    return decode(bytes);
+    return readString(in, Integer.MAX_VALUE);
   }
   }
   
   
   /** Read a UTF8 encoded string with a maximum size
   /** Read a UTF8 encoded string with a maximum size

+ 165 - 11
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java

@@ -23,6 +23,9 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
+import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
 import java.util.Map;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 
 
@@ -33,10 +36,11 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.SecureIOUtils.AlreadyExistsException;
 import org.apache.hadoop.io.SecureIOUtils.AlreadyExistsException;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
-
 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 sun.misc.Unsafe;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 
 
 /**
 /**
@@ -94,9 +98,6 @@ public class NativeIO {
 
 
     private static final Log LOG = LogFactory.getLog(NativeIO.class);
     private static final Log LOG = LogFactory.getLog(NativeIO.class);
 
 
-    @VisibleForTesting
-    public static CacheTracker cacheTracker = null;
-    
     private static boolean nativeLoaded = false;
     private static boolean nativeLoaded = false;
     private static boolean fadvisePossible = true;
     private static boolean fadvisePossible = true;
     private static boolean syncFileRangePossible = true;
     private static boolean syncFileRangePossible = true;
@@ -107,10 +108,71 @@ public class NativeIO {
 
 
     private static long cacheTimeout = -1;
     private static long cacheTimeout = -1;
 
 
-    public static interface CacheTracker {
-      public void fadvise(String identifier, long offset, long len, int flags);
+    private static CacheManipulator cacheManipulator = new CacheManipulator();
+
+    public static CacheManipulator getCacheManipulator() {
+      return cacheManipulator;
     }
     }
-    
+
+    public static void setCacheManipulator(CacheManipulator cacheManipulator) {
+      POSIX.cacheManipulator = cacheManipulator;
+    }
+
+    /**
+     * Used to manipulate the operating system cache.
+     */
+    @VisibleForTesting
+    public static class CacheManipulator {
+      public void mlock(String identifier, ByteBuffer buffer,
+          long len) throws IOException {
+        POSIX.mlock(buffer, len);
+      }
+
+      public long getMemlockLimit() {
+        return NativeIO.getMemlockLimit();
+      }
+
+      public long getOperatingSystemPageSize() {
+        return NativeIO.getOperatingSystemPageSize();
+      }
+
+      public void posixFadviseIfPossible(String identifier,
+        FileDescriptor fd, long offset, long len, int flags)
+            throws NativeIOException {
+        NativeIO.POSIX.posixFadviseIfPossible(identifier, fd, offset,
+            len, flags);
+      }
+
+      public boolean verifyCanMlock() {
+        return NativeIO.isAvailable();
+      }
+    }
+
+    /**
+     * A CacheManipulator used for testing which does not actually call mlock.
+     * This allows many tests to be run even when the operating system does not
+     * allow mlock, or only allows limited mlocking.
+     */
+    @VisibleForTesting
+    public static class NoMlockCacheManipulator extends CacheManipulator {
+      public void mlock(String identifier, ByteBuffer buffer,
+          long len) throws IOException {
+        LOG.info("mlocking " + identifier);
+      }
+
+      public long getMemlockLimit() {
+        return 1125899906842624L;
+      }
+
+      public long getOperatingSystemPageSize() {
+        return 4096;
+      }
+
+      public boolean verifyCanMlock() {
+        return true;
+      }
+    }
+
     static {
     static {
       if (NativeCodeLoader.isNativeCodeLoaded()) {
       if (NativeCodeLoader.isNativeCodeLoaded()) {
         try {
         try {
@@ -145,6 +207,12 @@ public class NativeIO {
       return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded;
       return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded;
     }
     }
 
 
+    private static void assertCodeLoaded() throws IOException {
+      if (!isAvailable()) {
+        throw new IOException("NativeIO was not loaded");
+      }
+    }
+
     /** Wrapper around open(2) */
     /** Wrapper around open(2) */
     public static native FileDescriptor open(String path, int flags, int mode) throws IOException;
     public static native FileDescriptor open(String path, int flags, int mode) throws IOException;
     /** Wrapper around fstat(2) */
     /** Wrapper around fstat(2) */
@@ -187,12 +255,9 @@ public class NativeIO {
      *
      *
      * @throws NativeIOException if there is an error with the syscall
      * @throws NativeIOException if there is an error with the syscall
      */
      */
-    public static void posixFadviseIfPossible(String identifier,
+    static void posixFadviseIfPossible(String identifier,
         FileDescriptor fd, long offset, long len, int flags)
         FileDescriptor fd, long offset, long len, int flags)
         throws NativeIOException {
         throws NativeIOException {
-      if (cacheTracker != null) {
-        cacheTracker.fadvise(identifier, offset, len, flags);
-      }
       if (nativeLoaded && fadvisePossible) {
       if (nativeLoaded && fadvisePossible) {
         try {
         try {
           posix_fadvise(fd, offset, len, flags);
           posix_fadvise(fd, offset, len, flags);
@@ -225,6 +290,66 @@ public class NativeIO {
       }
       }
     }
     }
 
 
+    static native void mlock_native(
+        ByteBuffer buffer, long len) throws NativeIOException;
+    static native void munlock_native(
+        ByteBuffer buffer, long len) throws NativeIOException;
+
+    /**
+     * Locks the provided direct ByteBuffer into memory, preventing it from
+     * swapping out. After a buffer is locked, future accesses will not incur
+     * a page fault.
+     * 
+     * See the mlock(2) man page for more information.
+     * 
+     * @throws NativeIOException
+     */
+    static void mlock(ByteBuffer buffer, long len)
+        throws IOException {
+      assertCodeLoaded();
+      if (!buffer.isDirect()) {
+        throw new IOException("Cannot mlock a non-direct ByteBuffer");
+      }
+      mlock_native(buffer, len);
+    }
+
+    /**
+     * Unlocks a locked direct ByteBuffer, allowing it to swap out of memory.
+     * This is a no-op if the ByteBuffer was not previously locked.
+     * 
+     * See the munlock(2) man page for more information.
+     * 
+     * @throws NativeIOException
+     */
+    public static void munlock(ByteBuffer buffer, long len)
+        throws IOException {
+      assertCodeLoaded();
+      if (!buffer.isDirect()) {
+        throw new IOException("Cannot munlock a non-direct ByteBuffer");
+      }
+      munlock_native(buffer, len);
+    }
+    
+    /**
+     * Unmaps the block from memory. See munmap(2).
+     *
+     * There isn't any portable way to unmap a memory region in Java.
+     * So we use the sun.nio method here.
+     * Note that unmapping a memory region could cause crashes if code
+     * continues to reference the unmapped code.  However, if we don't
+     * manually unmap the memory, we are dependent on the finalizer to
+     * do it, and we have no idea when the finalizer will run.
+     *
+     * @param buffer    The buffer to unmap.
+     */
+    public static void munmap(MappedByteBuffer buffer) {
+      if (buffer instanceof sun.nio.ch.DirectBuffer) {
+        sun.misc.Cleaner cleaner =
+            ((sun.nio.ch.DirectBuffer)buffer).cleaner();
+        cleaner.clean();
+      }
+    }
+
     /** Linux only methods used for getOwner() implementation */
     /** Linux only methods used for getOwner() implementation */
     private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException;
     private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException;
     private static native String getUserName(long uid) throws IOException;
     private static native String getUserName(long uid) throws IOException;
@@ -478,6 +603,35 @@ public class NativeIO {
   /** Initialize the JNI method ID and class ID cache */
   /** Initialize the JNI method ID and class ID cache */
   private static native void initNative();
   private static native void initNative();
 
 
+  /**
+   * Get the maximum number of bytes that can be locked into memory at any
+   * given point.
+   *
+   * @return 0 if no bytes can be locked into memory;
+   *         Long.MAX_VALUE if there is no limit;
+   *         The number of bytes that can be locked into memory otherwise.
+   */
+  static long getMemlockLimit() {
+    return isAvailable() ? getMemlockLimit0() : 0;
+  }
+
+  private static native long getMemlockLimit0();
+  
+  /**
+   * @return the operating system's page size.
+   */
+  static long getOperatingSystemPageSize() {
+    try {
+      Field f = Unsafe.class.getDeclaredField("theUnsafe");
+      f.setAccessible(true);
+      Unsafe unsafe = (Unsafe)f.get(null);
+      return unsafe.pageSize();
+    } catch (Throwable e) {
+      LOG.warn("Unable to get operating system page size.  Guessing 4096.", e);
+      return 4096;
+    }
+  }
+
   private static class CachedUid {
   private static class CachedUid {
     final long timestamp;
     final long timestamp;
     final String username;
     final String username;

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -1287,6 +1287,14 @@ public class UserGroupInformation {
     return null;
     return null;
   }
   }
 
 
+  public String getPrimaryGroupName() throws IOException {
+    String[] groups = getGroupNames();
+    if (groups.length == 0) {
+      throw new IOException("There is no primary group for UGI " + this);
+    }
+    return groups[0];
+  }
+
   /**
   /**
    * Get the user's full principal name.
    * Get the user's full principal name.
    * @return the user's full principal name.
    * @return the user's full principal name.

+ 373 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IntrusiveCollection.java

@@ -0,0 +1,373 @@
+/*
+ * 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.util;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Implements an intrusive doubly-linked list.
+ *
+ * An intrusive linked list is one in which the elements themselves are
+ * responsible for storing the pointers to previous and next elements.
+ * This can save a lot of memory if there are many elements in the list or
+ * many lists.
+ */
+@InterfaceAudience.Private
+public class IntrusiveCollection<E extends IntrusiveCollection.Element>
+    implements Collection<E> {
+  /**
+   * An element contained in this list.
+   *
+   * We pass the list itself as a parameter so that elements can belong to
+   * multiple lists.  (The element will need to store separate prev and next
+   * pointers for each.)
+   */
+  @InterfaceAudience.Private
+  public interface Element {
+    /**
+     * Insert this element into the list.  This is the first thing that will
+     * be called on the element.
+     */
+    void insertInternal(IntrusiveCollection<? extends Element> list,
+        Element prev, Element next);
+
+    /**
+     * Set the prev pointer of an element already in the list.
+     */
+    void setPrev(IntrusiveCollection<? extends Element> list, Element prev);
+
+    /**
+     * Set the next pointer of an element already in the list.
+     */
+    void setNext(IntrusiveCollection<? extends Element> list, Element next);
+
+    /**
+     * Remove an element from the list.  This is the last thing that will be
+     * called on an element.
+     */
+    void removeInternal(IntrusiveCollection<? extends Element> list);
+
+    /**
+     * Get the prev pointer of an element.
+     */
+    Element getPrev(IntrusiveCollection<? extends Element> list);
+
+    /**
+     * Get the next pointer of an element.
+     */
+    Element getNext(IntrusiveCollection<? extends Element> list);
+
+    /**
+     * Returns true if this element is in the provided list.
+     */
+    boolean isInList(IntrusiveCollection<? extends Element> list);
+  }
+
+  private Element root = new Element() {
+    // We keep references to the first and last elements for easy access.
+    Element first = this;
+    Element last = this;
+  
+    @Override
+    public void insertInternal(IntrusiveCollection<? extends Element> list,
+        Element prev, Element next) {
+      throw new RuntimeException("Can't insert root element");
+    }
+
+    @Override
+    public void setPrev(IntrusiveCollection<? extends Element> list,
+        Element prev) {
+      Preconditions.checkState(list == IntrusiveCollection.this);
+      last = prev;
+    }
+
+    @Override
+    public void setNext(IntrusiveCollection<? extends Element> list,
+        Element next) {
+      Preconditions.checkState(list == IntrusiveCollection.this);
+      first = next;
+    }
+  
+    @Override
+    public void removeInternal(IntrusiveCollection<? extends Element> list) {
+      throw new RuntimeException("Can't remove root element");
+    }
+    
+    @Override
+    public Element getNext(
+        IntrusiveCollection<? extends Element> list) {
+      Preconditions.checkState(list == IntrusiveCollection.this);
+      return first;
+    }
+  
+    @Override
+    public Element getPrev(
+        IntrusiveCollection<? extends Element> list) {
+      Preconditions.checkState(list == IntrusiveCollection.this);
+      return last;
+    }
+
+    @Override
+    public boolean isInList(IntrusiveCollection<? extends Element> list) {
+      return list == IntrusiveCollection.this;
+    }
+
+    @Override
+    public String toString() {
+      return "root"; // + IntrusiveCollection.this + "]";
+    }
+  };
+
+  private int size = 0;
+
+  /**
+   * An iterator over the intrusive collection.
+   *
+   * Currently, you can remove elements from the list using
+   * #{IntrusiveIterator#remove()}, but modifying the collection in other
+   * ways during the iteration is not supported.
+   */
+  public class IntrusiveIterator implements Iterator<E> {
+    Element cur;
+    Element next;
+
+    IntrusiveIterator() {
+      this.cur = root;
+      this.next = null;
+    }
+
+    @Override
+    public boolean hasNext() {
+      if (next == null) {
+        next = cur.getNext(IntrusiveCollection.this);
+      }
+      return next != root;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public E next() {
+      if (next == null) {
+        next = cur.getNext(IntrusiveCollection.this);
+      }
+      if (next == root) {
+        throw new NoSuchElementException();
+      }
+      cur = next;
+      next = null;
+      return (E)cur;
+    }
+
+    @Override
+    public void remove() {
+      if (cur == null) {
+        throw new IllegalStateException("Already called remove " +
+            "once on this element.");
+      }
+      next = removeElement(cur);
+      cur = null;
+    }
+  }
+  
+  private Element removeElement(Element elem) {
+    Element prev = elem.getPrev(IntrusiveCollection.this);
+    Element next = elem.getNext(IntrusiveCollection.this);
+    elem.removeInternal(IntrusiveCollection.this);
+    prev.setNext(IntrusiveCollection.this, next);
+    next.setPrev(IntrusiveCollection.this, prev);
+    size--;
+    return next;
+  }
+
+  /**
+   * Get an iterator over the list.  This can be used to remove elements.
+   * It is not safe to do concurrent modifications from other threads while
+   * using this iterator.
+   * 
+   * @return         The iterator.
+   */
+  public Iterator<E> iterator() {
+    return new IntrusiveIterator();
+  }
+
+  @Override
+  public int size() {
+    return size;
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return size == 0;
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    try {
+      Element element = (Element)o;
+      return element.isInList(this);
+    } catch (ClassCastException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public Object[] toArray() {
+    Object ret[] = new Object[size];
+    int i = 0;
+    for (Iterator<E> iter = iterator(); iter.hasNext(); ) {
+      ret[i++] = iter.next();
+    }
+    return ret;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T> T[] toArray(T[] array) {
+    if (array.length < size) {
+      return (T[])toArray();
+    } else {
+      int i = 0;
+      for (Iterator<E> iter = iterator(); iter.hasNext(); ) {
+        array[i++] = (T)iter.next();
+      }
+    }
+    return array;
+  }
+
+  /**
+   * Add an element to the end of the list.
+   * 
+   * @param elem     The new element to add.
+   */
+  @Override
+  public boolean add(E elem) {
+    if (elem == null) {
+      return false;
+    }
+    if (elem.isInList(this)) {
+      return false;
+    }
+    Element prev = root.getPrev(IntrusiveCollection.this);
+    prev.setNext(IntrusiveCollection.this, elem);
+    root.setPrev(IntrusiveCollection.this, elem);
+    elem.insertInternal(IntrusiveCollection.this, prev, root);
+    size++;
+    return true;
+  }
+
+  /**
+   * Add an element to the front of the list.
+   *
+   * @param elem     The new element to add.
+   */
+  public boolean addFirst(Element elem) {
+    if (elem == null) {
+      return false;
+    }
+    if (elem.isInList(this)) {
+      return false;
+    }
+    Element next = root.getNext(IntrusiveCollection.this);
+    next.setPrev(IntrusiveCollection.this, elem);
+    root.setNext(IntrusiveCollection.this, elem);
+    elem.insertInternal(IntrusiveCollection.this, root, next);
+    size++;
+    return true;
+  }
+
+  public static final Log LOG = LogFactory.getLog(IntrusiveCollection.class);
+
+  @Override
+  public boolean remove(Object o) {
+    try {
+      Element elem = (Element)o;
+      if (!elem.isInList(this)) {
+        return false;
+      }
+      removeElement(elem);
+      return true;
+    } catch (ClassCastException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public boolean containsAll(Collection<?> collection) {
+    for (Object o : collection) {
+      if (!contains(o)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public boolean addAll(Collection<? extends E> collection) {
+    boolean changed = false;
+    for (E elem : collection) {
+      if (add(elem)) {
+        changed = true;
+      }
+    }
+    return changed;
+  }
+
+  @Override
+  public boolean removeAll(Collection<?> collection) {
+    boolean changed = false;
+    for (Object elem : collection) {
+      if (remove(elem)) {
+        changed = true;
+      }
+    }
+    return changed;
+  }
+
+  @Override
+  public boolean retainAll(Collection<?> collection) {
+    boolean changed = false;
+    for (Iterator<E> iter = iterator();
+        iter.hasNext(); ) {
+      Element elem = iter.next();
+      if (!collection.contains(elem)) {
+        iter.remove();
+        changed = true;
+      }
+    }
+    return changed;
+  }
+
+  /**
+   * Remove all elements.
+   */
+  @Override
+  public void clear() {
+    for (Iterator<E> iter = iterator(); iter.hasNext(); ) {
+      iter.next();
+      iter.remove();
+    }
+  }
+}

+ 25 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.util;
 package org.apache.hadoop.util;
 
 
 import java.util.Comparator;
 import java.util.Comparator;
+import java.util.Iterator;
 import java.util.PriorityQueue;
 import java.util.PriorityQueue;
 
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -235,4 +236,28 @@ public class LightWeightCache<K, E extends K> extends LightWeightGSet<K, E> {
     }
     }
     return removed;
     return removed;
   }
   }
+
+  @Override
+  public Iterator<E> iterator() {
+    final Iterator<E> iter = super.iterator();
+    return new Iterator<E>() {
+      @Override
+      public boolean hasNext() {
+        return iter.hasNext();
+      }
+
+      @Override
+      public E next() {
+        return iter.next();
+      }
+
+      @Override
+      public void remove() {
+        // It would be tricky to support this because LightWeightCache#remove
+        // may evict multiple elements via evictExpiredEntries.
+        throw new UnsupportedOperationException("Remove via iterator is " +
+            "not supported for LightWeightCache");
+      }
+    };
+  }
 }
 }

+ 35 - 14
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java

@@ -246,10 +246,10 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
 
 
   private class SetIterator implements Iterator<E> {
   private class SetIterator implements Iterator<E> {
     /** The starting modification for fail-fast. */
     /** The starting modification for fail-fast. */
-    private final int startModification = modification;
+    private int iterModification = modification;
     /** The current index of the entry array. */
     /** The current index of the entry array. */
     private int index = -1;
     private int index = -1;
-    /** The next element to return. */
+    private LinkedElement cur = null;
     private LinkedElement next = nextNonemptyEntry();
     private LinkedElement next = nextNonemptyEntry();
 
 
     /** Find the next nonempty entry starting at (index + 1). */
     /** Find the next nonempty entry starting at (index + 1). */
@@ -258,30 +258,51 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
       return index < entries.length? entries[index]: null;
       return index < entries.length? entries[index]: null;
     }
     }
 
 
+    private void ensureNext() {
+      if (modification != iterModification) {
+        throw new ConcurrentModificationException("modification=" + modification
+            + " != iterModification = " + iterModification);
+      }
+      if (next != null) {
+        return;
+      }
+      if (cur == null) {
+        return;
+      }
+      next = cur.getNext();
+      if (next == null) {
+        next = nextNonemptyEntry();
+      }
+    }
+
     @Override
     @Override
     public boolean hasNext() {
     public boolean hasNext() {
+      ensureNext();
       return next != null;
       return next != null;
     }
     }
 
 
     @Override
     @Override
     public E next() {
     public E next() {
-      if (modification != startModification) {
-        throw new ConcurrentModificationException("modification=" + modification
-            + " != startModification = " + startModification);
+      ensureNext();
+      if (next == null) {
+        throw new IllegalStateException("There are no more elements");
       }
       }
-
-      final E e = convert(next);
-
-      //find the next element
-      final LinkedElement n = next.getNext();
-      next = n != null? n: nextNonemptyEntry();
-
-      return e;
+      cur = next;
+      next = null;
+      return convert(cur);
     }
     }
 
 
+    @SuppressWarnings("unchecked")
     @Override
     @Override
     public void remove() {
     public void remove() {
-      throw new UnsupportedOperationException("Remove is not supported.");
+      ensureNext();
+      if (cur == null) {
+        throw new IllegalStateException("There is no current element " +
+            "to remove");
+      }
+      LightWeightGSet.this.remove((K)cur);
+      iterModification++;
+      cur = null;
     }
     }
   }
   }
   
   

+ 79 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java

@@ -918,4 +918,83 @@ public class StringUtils {
     }
     }
     return str.toString();
     return str.toString();
   }
   }
+
+  /**
+   * From a list of command-line arguments, remove both an option and the 
+   * next argument.
+   *
+   * @param name  Name of the option to remove.  Example: -foo.
+   * @param args  List of arguments.
+   * @return      null if the option was not found; the value of the 
+   *              option otherwise.
+   * @throws IllegalArgumentException if the option's argument is not present
+   */
+  public static String popOptionWithArgument(String name, List<String> args)
+      throws IllegalArgumentException {
+    String val = null;
+    for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
+      String cur = iter.next();
+      if (cur.equals("--")) {
+        // stop parsing arguments when you see --
+        break;
+      } else if (cur.equals(name)) {
+        iter.remove();
+        if (!iter.hasNext()) {
+          throw new IllegalArgumentException("option " + name + " requires 1 " +
+              "argument.");
+        }
+        val = iter.next();
+        iter.remove();
+        break;
+      }
+    }
+    return val;
+  }
+  
+  /**
+   * From a list of command-line arguments, remove an option.
+   *
+   * @param name  Name of the option to remove.  Example: -foo.
+   * @param args  List of arguments.
+   * @return      true if the option was found and removed; false otherwise.
+   */
+  public static boolean popOption(String name, List<String> args) {
+    for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
+      String cur = iter.next();
+      if (cur.equals("--")) {
+        // stop parsing arguments when you see --
+        break;
+      } else if (cur.equals(name)) {
+        iter.remove();
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  /**
+   * From a list of command-line arguments, return the first non-option
+   * argument.  Non-option arguments are those which either come after 
+   * a double dash (--) or do not start with a dash.
+   *
+   * @param args  List of arguments.
+   * @return      The first non-option argument, or null if there were none.
+   */
+  public static String popFirstNonOption(List<String> args) {
+    for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
+      String cur = iter.next();
+      if (cur.equals("--")) {
+        if (!iter.hasNext()) {
+          return null;
+        }
+        cur = iter.next();
+        iter.remove();
+        return cur;
+      } else if (!cur.startsWith("-")) {
+        iter.remove();
+        return cur;
+      }
+    }
+    return null;
+  }
 }
 }

+ 87 - 2
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c

@@ -16,8 +16,6 @@
  * limitations under the License.
  * limitations under the License.
  */
  */
 
 
-#define _GNU_SOURCE
-
 #include "org_apache_hadoop.h"
 #include "org_apache_hadoop.h"
 #include "org_apache_hadoop_io_nativeio_NativeIO.h"
 #include "org_apache_hadoop_io_nativeio_NativeIO.h"
 
 
@@ -28,11 +26,15 @@
 #include <grp.h>
 #include <grp.h>
 #include <jni.h>
 #include <jni.h>
 #include <pwd.h>
 #include <pwd.h>
+#include <stdint.h>
 #include <stdio.h>
 #include <stdio.h>
 #include <stdlib.h>
 #include <stdlib.h>
 #include <string.h>
 #include <string.h>
+#include <sys/mman.h>
+#include <sys/resource.h>
 #include <sys/stat.h>
 #include <sys/stat.h>
 #include <sys/syscall.h>
 #include <sys/syscall.h>
+#include <sys/time.h>
 #include <sys/types.h>
 #include <sys/types.h>
 #include <unistd.h>
 #include <unistd.h>
 #include "config.h"
 #include "config.h"
@@ -360,6 +362,71 @@ Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_sync_1file_1range(
 #endif
 #endif
 }
 }
 
 
+#define CHECK_DIRECT_BUFFER_ADDRESS(buf) \
+  { \
+    if (!buf) { \
+      THROW(env, "java/lang/UnsupportedOperationException", \
+        "JNI access to direct buffers not available"); \
+      return; \
+    } \
+  }
+
+/**
+ * public static native void mlock_native(
+ *   ByteBuffer buffer, long offset);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_mlock_1native(
+  JNIEnv *env, jclass clazz,
+  jobject buffer, jlong len)
+{
+#ifdef UNIX
+  void* buf = (void*)(*env)->GetDirectBufferAddress(env, buffer);
+  PASS_EXCEPTIONS(env);
+
+  if (mlock(buf, len)) {
+    CHECK_DIRECT_BUFFER_ADDRESS(buf);
+    throw_ioe(env, errno);
+  }
+#endif
+
+#ifdef WINDOWS
+  THROW(env, "java/io/IOException",
+    "The function POSIX.mlock_native() is not supported on Windows");
+#endif
+}
+
+/**
+ * public static native void munlock_native(
+ *   ByteBuffer buffer, long offset);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_munlock_1native(
+  JNIEnv *env, jclass clazz,
+  jobject buffer, jlong len)
+{
+#ifdef UNIX
+  void* buf = (void*)(*env)->GetDirectBufferAddress(env, buffer);
+  PASS_EXCEPTIONS(env);
+
+  if (munlock(buf, len)) {
+    CHECK_DIRECT_BUFFER_ADDRESS(buf);
+    throw_ioe(env, errno);
+  }
+#endif
+
+#ifdef WINDOWS
+  THROW(env, "java/io/IOException",
+    "The function POSIX.munlock_native() is not supported on Windows");
+#endif
+}
+
 #ifdef __FreeBSD__
 #ifdef __FreeBSD__
 static int toFreeBSDFlags(int flags)
 static int toFreeBSDFlags(int flags)
 {
 {
@@ -924,6 +991,24 @@ done:
 #endif
 #endif
 }
 }
 
 
+JNIEXPORT jlong JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_getMemlockLimit0(
+JNIEnv *env, jclass clazz)
+{
+#ifdef WINDOWS
+  return 0;
+#else
+  struct rlimit rlim;
+  int rc = getrlimit(RLIMIT_MEMLOCK, &rlim);
+  if (rc != 0) {
+    throw_ioe(env, errno);
+    return 0;
+  }
+  return (rlim.rlim_cur == RLIM_INFINITY) ?
+    INT64_MAX : rlim.rlim_cur;
+#endif
+}
+
 /**
 /**
  * vim: sw=2: ts=2: et:
  * vim: sw=2: ts=2: et:
  */
  */

+ 108 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestBlockLocation.java

@@ -0,0 +1,108 @@
+/**
+ * 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import org.junit.Test;
+
+public class TestBlockLocation {
+
+  private static final String[] EMPTY_STR_ARRAY = new String[0];
+
+  private static void checkBlockLocation(final BlockLocation loc)
+      throws Exception {
+    checkBlockLocation(loc, 0, 0, false);
+  }
+
+  private static void checkBlockLocation(final BlockLocation loc,
+      final long offset, final long length, final boolean corrupt)
+      throws Exception {
+    checkBlockLocation(loc, EMPTY_STR_ARRAY, EMPTY_STR_ARRAY, EMPTY_STR_ARRAY,
+        EMPTY_STR_ARRAY, offset, length, corrupt);
+  }
+
+  private static void checkBlockLocation(final BlockLocation loc,
+      String[] names, String[] hosts, String[] cachedHosts,
+      String[] topologyPaths, final long offset, final long length,
+      final boolean corrupt) throws Exception {
+    assertNotNull(loc.getHosts());
+    assertNotNull(loc.getCachedHosts());
+    assertNotNull(loc.getNames());
+    assertNotNull(loc.getTopologyPaths());
+
+    assertArrayEquals(hosts, loc.getHosts());
+    assertArrayEquals(cachedHosts, loc.getCachedHosts());
+    assertArrayEquals(names, loc.getNames());
+    assertArrayEquals(topologyPaths, loc.getTopologyPaths());
+
+    assertEquals(offset, loc.getOffset());
+    assertEquals(length, loc.getLength());
+    assertEquals(corrupt, loc.isCorrupt());
+  }
+
+  /**
+   * Call all the constructors and verify the delegation is working properly
+   */
+  @Test(timeout = 5000)
+  public void testBlockLocationConstructors() throws Exception {
+    //
+    BlockLocation loc;
+    loc = new BlockLocation();
+    checkBlockLocation(loc);
+    loc = new BlockLocation(null, null, 1, 2);
+    checkBlockLocation(loc, 1, 2, false);
+    loc = new BlockLocation(null, null, null, 1, 2);
+    checkBlockLocation(loc, 1, 2, false);
+    loc = new BlockLocation(null, null, null, 1, 2, true);
+    checkBlockLocation(loc, 1, 2, true);
+    loc = new BlockLocation(null, null, null, null, 1, 2, true);
+    checkBlockLocation(loc, 1, 2, true);
+  }
+
+  /**
+   * Call each of the setters and verify
+   */
+  @Test(timeout = 5000)
+  public void testBlockLocationSetters() throws Exception {
+    BlockLocation loc;
+    loc = new BlockLocation();
+    // Test that null sets the empty array
+    loc.setHosts(null);
+    loc.setCachedHosts(null);
+    loc.setNames(null);
+    loc.setTopologyPaths(null);
+    checkBlockLocation(loc);
+    // Test that not-null gets set properly
+    String[] names = new String[] { "name" };
+    String[] hosts = new String[] { "host" };
+    String[] cachedHosts = new String[] { "cachedHost" };
+    String[] topologyPaths = new String[] { "path" };
+    loc.setNames(names);
+    loc.setHosts(hosts);
+    loc.setCachedHosts(cachedHosts);
+    loc.setTopologyPaths(topologyPaths);
+    loc.setOffset(1);
+    loc.setLength(2);
+    loc.setCorrupt(true);
+    checkBlockLocation(loc, names, hosts, cachedHosts, topologyPaths, 1, 2,
+        true);
+  }
+}

+ 61 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java

@@ -24,6 +24,9 @@ import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.IOException;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileChannel.MapMode;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
@@ -32,6 +35,7 @@ import java.util.List;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+
 import static org.junit.Assume.*;
 import static org.junit.Assume.*;
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
@@ -45,6 +49,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
 public class TestNativeIO {
 public class TestNativeIO {
@@ -563,4 +568,60 @@ public class TestNativeIO {
 
 
     FileUtils.deleteQuietly(TEST_DIR);
     FileUtils.deleteQuietly(TEST_DIR);
   }
   }
+
+  @Test(timeout=10000)
+  public void testMlock() throws Exception {
+    assumeTrue(NativeIO.isAvailable());
+    assumeTrue(Shell.LINUX);
+    final File TEST_FILE = new File(new File(
+        System.getProperty("test.build.data","build/test/data")),
+        "testMlockFile");
+    final int BUF_LEN = 12289;
+    byte buf[] = new byte[BUF_LEN];
+    int bufSum = 0;
+    for (int i = 0; i < buf.length; i++) {
+      buf[i] = (byte)(i % 60);
+      bufSum += buf[i];
+    }
+    FileOutputStream fos = new FileOutputStream(TEST_FILE);
+    try {
+      fos.write(buf);
+      fos.getChannel().force(true);
+    } finally {
+      fos.close();
+    }
+    
+    FileInputStream fis = null;
+    FileChannel channel = null;
+    try {
+      // Map file into memory
+      fis = new FileInputStream(TEST_FILE);
+      channel = fis.getChannel();
+      long fileSize = channel.size();
+      MappedByteBuffer mapbuf = channel.map(MapMode.READ_ONLY, 0, fileSize);
+      // mlock the buffer
+      NativeIO.POSIX.mlock(mapbuf, fileSize);
+      // Read the buffer
+      int sum = 0;
+      for (int i=0; i<fileSize; i++) {
+        sum += mapbuf.get(i);
+      }
+      assertEquals("Expected sums to be equal", bufSum, sum);
+      // munlock the buffer
+      NativeIO.POSIX.munlock(mapbuf, fileSize);
+    } finally {
+      if (channel != null) {
+        channel.close();
+      }
+      if (fis != null) {
+        fis.close();
+      }
+    }
+  }
+
+  @Test(timeout=10000)
+  public void testGetMemlockLimit() throws Exception {
+    assumeTrue(NativeIO.isAvailable());
+    NativeIO.getMemlockLimit();
+  }
 }
 }

+ 110 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java

@@ -0,0 +1,110 @@
+/**
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
+import org.junit.Assert;
+import org.junit.Test;
+
+/** Testing {@link LightWeightGSet} */
+public class TestLightWeightGSet {
+  public static final Log LOG = LogFactory.getLog(TestLightWeightGSet.class);
+
+  private static ArrayList<Integer> getRandomList(int length, int randomSeed) {
+    Random random = new Random(randomSeed);
+    ArrayList<Integer> list = new ArrayList<Integer>(length);
+    for (int i = 0; i < length; i++) {
+      list.add(random.nextInt());
+    }
+    return list;
+  }
+  
+  private static class TestElement implements LightWeightGSet.LinkedElement {
+    private final int val;
+    private LinkedElement next;
+
+    TestElement(int val) {
+      this.val = val;
+      this.next = null;
+    }
+    
+    public int getVal() {
+      return val;
+    }
+
+    @Override
+    public void setNext(LinkedElement next) {
+      this.next = next;
+    }
+
+    @Override
+    public LinkedElement getNext() {
+      return next;
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testRemoveAllViaIterator() {
+    ArrayList<Integer> list = getRandomList(100, 123);
+    LightWeightGSet<TestElement, TestElement> set =
+        new LightWeightGSet<TestElement, TestElement>(16);
+    for (Integer i : list) {
+      set.put(new TestElement(i));
+    }
+    for (Iterator<TestElement> iter = set.iterator();
+        iter.hasNext(); ) {
+      iter.next();
+      iter.remove();
+    }
+    Assert.assertEquals(0, set.size());
+  }
+
+  @Test(timeout=60000)
+  public void testRemoveSomeViaIterator() {
+    ArrayList<Integer> list = getRandomList(100, 123);
+    LightWeightGSet<TestElement, TestElement> set =
+        new LightWeightGSet<TestElement, TestElement>(16);
+    for (Integer i : list) {
+      set.put(new TestElement(i));
+    }
+    long sum = 0;
+    for (Iterator<TestElement> iter = set.iterator();
+        iter.hasNext(); ) {
+      sum += iter.next().getVal();
+    }
+    long mode = sum / set.size();
+    LOG.info("Removing all elements above " + mode);
+    for (Iterator<TestElement> iter = set.iterator();
+        iter.hasNext(); ) {
+      int item = iter.next().getVal();
+      if (item > mode) {
+        iter.remove();
+      }
+    }
+    for (Iterator<TestElement> iter = set.iterator();
+        iter.hasNext(); ) {
+      Assert.assertTrue(iter.next().getVal() <= mode);
+    }
+  }
+}

+ 208 - 1
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -36,6 +36,8 @@ Release 2.4.0 - UNRELEASED
 
 
     HDFS-5703. Add support for HTTPS and swebhdfs to HttpFS. (tucu)
     HDFS-5703. Add support for HTTPS and swebhdfs to HttpFS. (tucu)
 
 
+    HDFS-4949. Centralized cache management in HDFS (wang and cmccabe)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
@@ -426,6 +428,211 @@ Release 2.4.0 - UNRELEASED
 
 
     HDFS-5667. Include DatanodeStorage in StorageReport. (Arpit Agarwal)
     HDFS-5667. Include DatanodeStorage in StorageReport. (Arpit Agarwal)
 
 
+  BREAKDOWN OF HDFS-4949 SUBTASKS AND RELATED JIRAS
+
+    HDFS-5049.  Add JNI mlock support.  (Andrew Wang via Colin Patrick McCabe)
+
+    HDFS-5051.  Propagate cache status information from the DataNode to the
+    NameNode  (Andrew Wang via Colin Patrick McCabe)
+
+    HDFS-5052. Add cacheRequest/uncacheRequest support to NameNode.
+    (Contributed by Colin Patrick McCabe.)
+
+    HDFS-5050.  Add DataNode support for mlock and munlock  (contributed by
+    Andrew Wang)
+
+    HDFS-5141. Add cache status information to datanode heartbeat. (Contributed
+    by Andrew Wang)
+
+    HDFS-5121.  Add RPCs for creating and manipulating cache pools.
+    (Contributed by Colin Patrick McCabe)
+
+    HDFS-5163. Miscellaneous cache pool RPC fixes (Contributed by Colin Patrick
+    McCabe)
+
+    HDFS-5169. hdfs.c: translateZCRException: null pointer deref when
+    translating some exceptions (Contributed by Colin Patrick McCabe)
+
+    HDFS-5120. Add command-line support for manipulating cache pools. (cmccabe)
+
+    HDFS-5158. Add command-line support for manipulating cache directives.
+    (cmccabe)
+
+    HDFS-5198. NameNodeRpcServer must not send back DNA_FINALIZE in reply to a
+    cache report. (cmccabe)
+
+    HDFS-5195. Prevent passing null pointer to mlock and munlock. Contributed
+    by Chris Nauroth.
+
+    HDFS-5053. NameNode should invoke DataNode APIs to coordinate caching.
+    (Andrew Wang)
+
+    HDFS-5201. NativeIO: consolidate getrlimit into NativeIO#getMemlockLimit.
+    (Contributed by Colin Patrick McCabe)
+
+    HDFS-5197. Document dfs.cachereport.intervalMsec in hdfs-default.xml.
+    Contributed by Chris Nauroth.
+
+    HDFS-5210. Fix some failing unit tests on HDFS-4949 branch. (Contributed by
+    Andrew Wang)
+
+    HDFS-5213. Separate PathBasedCacheEntry and PathBasedCacheDirectiveWithId.
+    Contributed by Colin Patrick McCabe.
+
+    HDFS-5236. Change PathBasedCacheDirective APIs to be a single value rather
+    than batch. (Contributed by Andrew Wang)
+
+    HDFS-5119. Persist CacheManager state in the edit log. (Contributed by
+    Andrew Wang)
+
+    HDFS-5190. Move cache pool related CLI commands to CacheAdmin. (Contributed
+    by Andrew Wang)
+
+    HDFS-5309. Fix failing caching unit tests. (Andrew Wang)
+
+    HDFS-5314.  Do not expose CachePool type in AddCachePoolOp (Colin Patrick
+    McCabe)
+
+    HDFS-5304. Expose if a block replica is cached in getFileBlockLocations.
+    (Contributed by Andrew Wang)
+
+    HDFS-5224. Refactor PathBasedCache* methods to use a Path rather than a
+    String. Contributed by Chris Nauroth.
+
+    HDFS-5348. Fix error message when dfs.datanode.max.locked.memory is
+    improperly configured. (Contributed by Colin Patrick McCabe)
+
+    HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only (cmccabe)
+
+    HDFS-5358. Add replication field to PathBasedCacheDirective. (Contributed
+    by Colin Patrick McCabe)
+
+    HDFS-5359. Allow LightWeightGSet#Iterator to remove elements. (Contributed
+    by Colin Patrick McCabe)
+
+    HDFS-5373. hdfs cacheadmin -addDirective short usage does not mention
+    -replication parameter. Contributed by Chris Nauroth.
+
+    HDFS-5096. Automatically cache new data added to a cached path (contributed
+    by Colin Patrick McCabe)
+
+    HDFS-5383. fix broken caching unit tests (Andrew Wang)
+
+    HDFS-5388. Loading fsimage fails to find cache pools during namenode
+    startup (Chris Nauroth via Colin Patrick McCabe)
+
+    HDFS-5203. Concurrent clients that add a cache directive on the same path
+    may prematurely uncache each other.  (Chris Nauroth via Colin Patrick McCabe)
+
+    HDFS-5378. In CacheReport, don't send genstamp and length on the wire
+    (Contributed by Colin Patrick McCabe)
+
+    HDFS-5385. Caching RPCs are AtMostOnce, but do not persist client ID and
+    call ID to edit log.  (Chris Nauroth via Colin Patrick McCabe)
+
+    HDFS-5404 Resolve regressions in Windows compatibility on HDFS-4949 branch.
+    Contributed by Chris Nauroth.
+
+    HDFS-5405. Fix possible RetryCache hang for caching RPC handlers in
+    FSNamesystem. (Contributed by Andrew Wang)
+
+    HDFS-5419. Fixup test-patch.sh warnings on HDFS-4949 branch. (wang)
+
+    HDFS-5386. Add feature documentation for datanode caching. Contributed by
+    Colin Patrick McCabe.
+
+    HDFS-5468. CacheAdmin help command does not recognize commands  (Stephen
+    Chu via Colin Patrick McCabe)
+
+    HDFS-5326. add modifyDirective to cacheAdmin (cmccabe)
+
+    HDFS-5394: Fix race conditions in DN caching and uncaching (cmccabe)
+
+    HDFS-5320. Add datanode caching metrics. Contributed by Andrew Wang.
+
+    HDFS-5482. DistributedFileSystem#listPathBasedCacheDirectives must support
+    relative paths. Contributed by Colin Patrick McCabe.
+
+    HDFS-5471. CacheAdmin -listPools fails when user lacks permissions to view
+    all pools (Andrew Wang via Colin Patrick McCabe)
+
+    HDFS-5450. better API for getting the cached blocks locations. Contributed
+    by Andrew Wang.
+
+    HDFS-5485. add command-line support for modifyDirective (cmccabe)
+
+    HDFS-5366. recaching improvements (cmccabe)
+
+    HDFS-5520. loading cache path directives from edit log doesnt update
+    nextEntryId (cmccabe)
+
+    HDFS-5512. CacheAdmin -listPools fails with NPE when user lacks permissions
+    to view all pools (awang via cmccabe)
+
+    HDFS-5513. CacheAdmin commands fail when using . as the path. Contributed
+    by Andrew Wang.
+
+    HDFS-5511. improve CacheManipulator interface to allow better unit testing
+    (cmccabe)
+
+    HDFS-5451. Add byte and file statistics to PathBasedCacheEntry. Contributed
+    by Colin Patrick McCabe.
+
+    HDFS-5473. Consistent naming of user-visible caching classes and methods
+    (cmccabe)
+
+    HDFS-5543. Fix narrow race condition in TestPathBasedCacheRequests
+    (cmccabe)
+
+    HDFS-5565. CacheAdmin help should match against non-dashed commands (wang
+    via cmccabe)
+
+    HDFS-5556. Add some more NameNode cache statistics, cache pool stats
+    (cmccabe)
+
+    HDFS-5562. TestCacheDirectives and TestFsDatasetCache should stub out
+    native mlock. Contributed by Colin Patrick McCabe and Akira Ajisaka.
+
+    HDFS-5430. Support TTL on CacheDirectives. Contributed by Andrew Wang.
+
+    HDFS-5555. CacheAdmin commands fail when first listed NameNode is in
+    Standby (jxiang via cmccabe)
+
+    HDFS-5626. dfsadmin report shows incorrect values (cmccabe)
+
+    HDFS-5630. Hook up cache directive and pool usage statistics. (wang)
+
+    HDFS-5665. Remove the unnecessary writeLock while initializing CacheManager
+    in FsNameSystem Ctor. (Uma Maheswara Rao G via Andrew Wang)
+
+    HDFS-5431. Support cachepool-based limit management in path-based caching.
+    (awang via cmccabe)
+
+    HDFS-5679. TestCacheDirectives should handle the case where native code is
+    not available. (wang)
+
+    HDFS-5636. Enforce a max TTL per cache pool (awang via cmccabe)
+
+    HDFS-5701. Fix the CacheAdmin -addPool -maxTtl option name. Contributed by
+    Stephen Chu.
+
+    HDFS-5708. The CacheManager throws a NPE in the DataNode logs when
+    processing cache reports that refer to a block not known to the BlockManager.
+    Contributed by Colin Patrick McCabe.
+
+    HDFS-5659. dfsadmin -report doesn't output cache information properly.
+    Contributed by Andrew Wang.
+
+    HDFS-5651. Remove dfs.namenode.caching.enabled and improve CRM locking.
+    Contributed by Colin Patrick McCabe.
+
+    HDFS-5589. Namenode loops caching and uncaching when data should be
+    uncached. (awang via cmccabe)
+
+    HDFS-5724. modifyCacheDirective logging audit log command wrongly as
+    addCacheDirective (Uma Maheswara Rao G via Colin Patrick McCabe)
+
+
 Release 2.3.0 - UNRELEASED
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -799,7 +1006,7 @@ Release 2.1.1-beta - 2013-09-23
     HDFS-5091. Support for spnego keytab separate from the JournalNode keytab 
     HDFS-5091. Support for spnego keytab separate from the JournalNode keytab 
     for secure HA. (jing9)
     for secure HA. (jing9)
 
 
-    HDFS-5051. nn fails to download checkpointed image from snn in some
+    HDFS-5055. nn fails to download checkpointed image from snn in some
     setups. (Vinay and suresh via suresh)
     setups. (Vinay and suresh via suresh)
 
 
     HDFS-4898. BlockPlacementPolicyWithNodeGroup.chooseRemoteRack() fails to
     HDFS-4898. BlockPlacementPolicyWithNodeGroup.chooseRemoteRack() fails to

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -346,4 +346,20 @@
       <Method name="create" />
       <Method name="create" />
       <Bug pattern="UL_UNRELEASED_LOCK" />
       <Bug pattern="UL_UNRELEASED_LOCK" />
     </Match>
     </Match>
+    <!-- Manually verified to be okay, we want to throw away the top bit here -->
+    <Match>
+      <Class name="org.apache.hadoop.hdfs.server.namenode.CachedBlock" />
+      <Method name="getReplication" />
+      <Bug pattern="ICAST_QUESTIONABLE_UNSIGNED_RIGHT_SHIFT" />
+    </Match>
+    <Match>
+      <Class name="org.apache.hadoop.hdfs.protocol.CacheDirective" />
+      <Method name="insertInternal" />
+      <Bug pattern="BC_UNCONFIRMED_CAST" />
+    </Match>
+    <Match>
+      <Class name="org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor" />
+      <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
+    </Match>
+
  </FindBugsFilter>
  </FindBugsFilter>

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -59,6 +59,7 @@ function print_usage(){
   echo "						Use -help to see options"
   echo "						Use -help to see options"
   echo "  portmap              run a portmap service"
   echo "  portmap              run a portmap service"
   echo "  nfs3                 run an NFS version 3 gateway"
   echo "  nfs3                 run an NFS version 3 gateway"
+  echo "  cacheadmin           configure the HDFS cache"
   echo ""
   echo ""
   echo "Most commands print help when invoked w/o parameters."
   echo "Most commands print help when invoked w/o parameters."
 }
 }
@@ -155,6 +156,8 @@ elif [ "$COMMAND" = "portmap" ] ; then
   CLASS=org.apache.hadoop.portmap.Portmap
   CLASS=org.apache.hadoop.portmap.Portmap
 elif [ "$COMMAND" = "nfs3" ] ; then
 elif [ "$COMMAND" = "nfs3" ] ; then
   CLASS=org.apache.hadoop.hdfs.nfs.nfs3.Nfs3
   CLASS=org.apache.hadoop.hdfs.nfs.nfs3.Nfs3
+elif [ "$COMMAND" = "cacheadmin" ] ; then
+  CLASS=org.apache.hadoop.hdfs.tools.CacheAdmin
 else
 else
   CLASS="$COMMAND"
   CLASS="$COMMAND"
 fi
 fi

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

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java

@@ -37,8 +37,7 @@ public class HdfsBlockLocation extends BlockLocation {
   public HdfsBlockLocation(BlockLocation loc, LocatedBlock block) 
   public HdfsBlockLocation(BlockLocation loc, LocatedBlock block) 
       throws IOException {
       throws IOException {
     // Initialize with data from passed in BlockLocation
     // Initialize with data from passed in BlockLocation
-    super(loc.getNames(), loc.getHosts(), loc.getTopologyPaths(), 
-        loc.getOffset(), loc.getLength(), loc.isCorrupt());
+    super(loc);
     this.block = block;
     this.block = block;
   }
   }
   
   

+ 74 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -87,6 +87,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;
@@ -100,6 +101,7 @@ 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;
@@ -109,6 +111,11 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 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.CacheDirectiveIterator;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolIterator;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@@ -117,6 +124,7 @@ 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;
@@ -2312,7 +2320,73 @@ public class DFSClient implements java.io.Closeable {
       throw re.unwrapRemoteException();
       throw re.unwrapRemoteException();
     }
     }
   }
   }
+
+  public long addCacheDirective(
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+    checkOpen();
+    try {
+      return namenode.addCacheDirective(info, flags);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+  
+  public void modifyCacheDirective(
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+    checkOpen();
+    try {
+      namenode.modifyCacheDirective(info, flags);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+
+  public void removeCacheDirective(long id)
+      throws IOException {
+    checkOpen();
+    try {
+      namenode.removeCacheDirective(id);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
   
   
+  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+      CacheDirectiveInfo filter) throws IOException {
+    return new CacheDirectiveIterator(namenode, filter);
+  }
+
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    checkOpen();
+    try {
+      namenode.addCachePool(info);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    checkOpen();
+    try {
+      namenode.modifyCachePool(info);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+
+  public void removeCachePool(String poolName) throws IOException {
+    checkOpen();
+    try {
+      namenode.removeCachePool(poolName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+
+  public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
+    return new CachePoolIterator(namenode);
+  }
+
   /**
   /**
    * Save namespace image.
    * Save namespace image.
    * 
    * 

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

@@ -104,6 +104,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT = false;
   public static final boolean DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT = false;
   public static final String  DFS_DATANODE_USE_DN_HOSTNAME = "dfs.datanode.use.datanode.hostname";
   public static final String  DFS_DATANODE_USE_DN_HOSTNAME = "dfs.datanode.use.datanode.hostname";
   public static final boolean DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT = false;
   public static final boolean DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT = false;
+  public static final String  DFS_DATANODE_MAX_LOCKED_MEMORY_KEY = "dfs.datanode.max.locked.memory";
+  public static final long    DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT = 0;
+  public static final String  DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY = "dfs.datanode.fsdatasetcache.max.threads.per.volume";
+  public static final int     DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT = 4;
+  public static final String  DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT =
+    "dfs.namenode.path.based.cache.block.map.allocation.percent";
+  public static final float    DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT_DEFAULT = 0.25f;
 
 
   public static final String  DFS_NAMENODE_HTTP_PORT_KEY = "dfs.http.port";
   public static final String  DFS_NAMENODE_HTTP_PORT_KEY = "dfs.http.port";
   public static final int     DFS_NAMENODE_HTTP_PORT_DEFAULT = 50070;
   public static final int     DFS_NAMENODE_HTTP_PORT_DEFAULT = 50070;
@@ -210,6 +217,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
 
 
   public static final String DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY = "dfs.namenode.datanode.registration.ip-hostname-check";
   public static final String DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY = "dfs.namenode.datanode.registration.ip-hostname-check";
   public static final boolean DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT = true;
   public static final boolean DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT = true;
+  
+  public static final String  DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES =
+      "dfs.namenode.list.cache.pools.num.responses";
+  public static final int     DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT = 100;
+  public static final String  DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES =
+      "dfs.namenode.list.cache.directives.num.responses";
+  public static final int     DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES_DEFAULT = 100;
+  public static final String  DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS =
+      "dfs.namenode.path.based.cache.refresh.interval.ms";
+  public static final long    DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT = 300000L;
 
 
   // Whether to enable datanode's stale state detection and usage for reads
   // Whether to enable datanode's stale state detection and usage for reads
   public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
   public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
@@ -335,6 +352,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT = true;
   public static final boolean DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT = true;
   public static final String  DFS_HEARTBEAT_INTERVAL_KEY = "dfs.heartbeat.interval";
   public static final String  DFS_HEARTBEAT_INTERVAL_KEY = "dfs.heartbeat.interval";
   public static final long    DFS_HEARTBEAT_INTERVAL_DEFAULT = 3;
   public static final long    DFS_HEARTBEAT_INTERVAL_DEFAULT = 3;
+  public static final String  DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS = "dfs.namenode.path.based.cache.retry.interval.ms";
+  public static final long    DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS_DEFAULT = 60000L;
   public static final String  DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY = "dfs.namenode.decommission.interval";
   public static final String  DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY = "dfs.namenode.decommission.interval";
   public static final int     DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT = 30;
   public static final int     DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT = 30;
   public static final String  DFS_NAMENODE_DECOMMISSION_NODES_PER_INTERVAL_KEY = "dfs.namenode.decommission.nodes.per.interval";
   public static final String  DFS_NAMENODE_DECOMMISSION_NODES_PER_INTERVAL_KEY = "dfs.namenode.decommission.nodes.per.interval";
@@ -378,6 +397,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long    DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT = 60 * 60 * 1000;
   public static final long    DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT = 60 * 60 * 1000;
   public static final String  DFS_BLOCKREPORT_INITIAL_DELAY_KEY = "dfs.blockreport.initialDelay";
   public static final String  DFS_BLOCKREPORT_INITIAL_DELAY_KEY = "dfs.blockreport.initialDelay";
   public static final int     DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT = 0;
   public static final int     DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT = 0;
+  public static final String  DFS_CACHEREPORT_INTERVAL_MSEC_KEY = "dfs.cachereport.intervalMsec";
+  public static final long    DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT = 10 * 1000;
   public static final String  DFS_BLOCK_INVALIDATE_LIMIT_KEY = "dfs.block.invalidate.limit";
   public static final String  DFS_BLOCK_INVALIDATE_LIMIT_KEY = "dfs.block.invalidate.limit";
   public static final int     DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT = 1000;
   public static final int     DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT = 1000;
   public static final String  DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";
   public static final String  DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";

+ 79 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -41,13 +41,15 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.security.SecureRandom;
 import java.security.SecureRandom;
+import java.text.SimpleDateFormat;
 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.Comparator;
 import java.util.Comparator;
-import java.util.HashMap;
+import java.util.Date;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Map;
 import java.util.Random;
 import java.util.Random;
 import java.util.Set;
 import java.util.Set;
@@ -449,7 +451,13 @@ public class DFSUtil {
                                      locations[hCnt].getNetworkLocation());
                                      locations[hCnt].getNetworkLocation());
         racks[hCnt] = node.toString();
         racks[hCnt] = node.toString();
       }
       }
-      blkLocations[idx] = new BlockLocation(xferAddrs, hosts, racks,
+      DatanodeInfo[] cachedLocations = blk.getCachedLocations();
+      String[] cachedHosts = new String[cachedLocations.length];
+      for (int i=0; i<cachedLocations.length; i++) {
+        cachedHosts[i] = cachedLocations[i].getHostName();
+      }
+      blkLocations[idx] = new BlockLocation(xferAddrs, hosts, cachedHosts,
+                                            racks,
                                             blk.getStartOffset(),
                                             blk.getStartOffset(),
                                             blk.getBlockSize(),
                                             blk.getBlockSize(),
                                             blk.isCorrupt());
                                             blk.isCorrupt());
@@ -1595,4 +1603,73 @@ public class DFSUtil {
     }
     }
     return builder;
     return builder;
   }
   }
+
+  /**
+   * Converts a Date into an ISO-8601 formatted datetime string.
+   */
+  public static String dateToIso8601String(Date date) {
+    SimpleDateFormat df =
+        new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ", Locale.ENGLISH);
+    return df.format(date);
+  }
+
+  /**
+   * Converts a time duration in milliseconds into DDD:HH:MM:SS format.
+   */
+  public static String durationToString(long durationMs) {
+    boolean negative = false;
+    if (durationMs < 0) {
+      negative = true;
+      durationMs = -durationMs;
+    }
+    // Chop off the milliseconds
+    long durationSec = durationMs / 1000;
+    final int secondsPerMinute = 60;
+    final int secondsPerHour = 60*60;
+    final int secondsPerDay = 60*60*24;
+    final long days = durationSec / secondsPerDay;
+    durationSec -= days * secondsPerDay;
+    final long hours = durationSec / secondsPerHour;
+    durationSec -= hours * secondsPerHour;
+    final long minutes = durationSec / secondsPerMinute;
+    durationSec -= minutes * secondsPerMinute;
+    final long seconds = durationSec;
+    final long milliseconds = durationMs % 1000;
+    String format = "%03d:%02d:%02d:%02d.%03d";
+    if (negative)  {
+      format = "-" + format;
+    }
+    return String.format(format, days, hours, minutes, seconds, milliseconds);
+  }
+
+  /**
+   * Converts a relative time string into a duration in milliseconds.
+   */
+  public static long parseRelativeTime(String relTime) throws IOException {
+    if (relTime.length() < 2) {
+      throw new IOException("Unable to parse relative time value of " + relTime
+          + ": too short");
+    }
+    String ttlString = relTime.substring(0, relTime.length()-1);
+    long ttl;
+    try {
+      ttl = Long.parseLong(ttlString);
+    } catch (NumberFormatException e) {
+      throw new IOException("Unable to parse relative time value of " + relTime
+          + ": " + ttlString + " is not a number");
+    }
+    if (relTime.endsWith("s")) {
+      // pass
+    } else if (relTime.endsWith("m")) {
+      ttl *= 60;
+    } else if (relTime.endsWith("h")) {
+      ttl *= 60*60;
+    } else if (relTime.endsWith("d")) {
+      ttl *= 60*60*24;
+    } else {
+      throw new IOException("Unable to parse relative time value of " + relTime
+          + ": unknown time unit " + relTime.charAt(relTime.length() - 1));
+    }
+    return ttl*1000;
+  }
 }
 }

+ 159 - 0
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;
@@ -57,6 +58,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 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.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -66,6 +70,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
@@ -79,6 +84,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 
 
 
 /****************************************************************
 /****************************************************************
@@ -1627,5 +1633,158 @@ 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.
+   * 
+   * @param info Information about a directive to add.
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @return the ID of the directive that was created.
+   * @throws IOException if the directive could not be added
+   */
+  public long addCacheDirective(
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+    Preconditions.checkNotNull(info.getPath());
+    Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
+        makeQualified(getUri(), getWorkingDirectory());
+    return dfs.addCacheDirective(
+        new CacheDirectiveInfo.Builder(info).
+            setPath(path).
+            build(),
+        flags);
+  }
+
+  /**
+   * @see {@link #modifyCacheDirective(CacheDirectiveInfo, EnumSet)}
+   */
+  public void modifyCacheDirective(CacheDirectiveInfo info) throws IOException {
+    modifyCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
+  }
+
+  /**
+   * Modify a CacheDirective.
+   * 
+   * @param info Information about the directive to modify. You must set the ID
+   *          to indicate which CacheDirective you want to modify.
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @throws IOException if the directive could not be modified
+   */
+  public void modifyCacheDirective(
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+    if (info.getPath() != null) {
+      info = new CacheDirectiveInfo.Builder(info).
+          setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
+              makeQualified(getUri(), getWorkingDirectory())).build();
+    }
+    dfs.modifyCacheDirective(info, flags);
+  }
+
+  /**
+   * Remove a CacheDirectiveInfo.
+   * 
+   * @param id identifier of the CacheDirectiveInfo to remove
+   * @throws IOException if the directive could not be removed
+   */
+  public void removeCacheDirective(long id)
+      throws IOException {
+    dfs.removeCacheDirective(id);
+  }
   
   
+  /**
+   * List cache directives.  Incrementally fetches results from the server.
+   * 
+   * @param filter Filter parameters to use when listing the directives, null to
+   *               list all directives visible to us.
+   * @return A RemoteIterator which returns CacheDirectiveInfo objects.
+   */
+  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+      CacheDirectiveInfo filter) throws IOException {
+    if (filter == null) {
+      filter = new CacheDirectiveInfo.Builder().build();
+    }
+    if (filter.getPath() != null) {
+      filter = new CacheDirectiveInfo.Builder(filter).
+          setPath(new Path(getPathName(fixRelativePart(filter.getPath())))).
+          build();
+    }
+    final RemoteIterator<CacheDirectiveEntry> iter =
+        dfs.listCacheDirectives(filter);
+    return new RemoteIterator<CacheDirectiveEntry>() {
+      @Override
+      public boolean hasNext() throws IOException {
+        return iter.hasNext();
+      }
+
+      @Override
+      public CacheDirectiveEntry next() throws IOException {
+        // Although the paths we get back from the NameNode should always be
+        // absolute, we call makeQualified to add the scheme and authority of
+        // this DistributedFilesystem.
+        CacheDirectiveEntry desc = iter.next();
+        CacheDirectiveInfo info = desc.getInfo();
+        Path p = info.getPath().makeQualified(getUri(), getWorkingDirectory());
+        return new CacheDirectiveEntry(
+            new CacheDirectiveInfo.Builder(info).setPath(p).build(),
+            desc.getStats());
+      }
+    };
+  }
+
+  /**
+   * Add a cache pool.
+   *
+   * @param info
+   *          The request to add a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    CachePoolInfo.validate(info);
+    dfs.addCachePool(info);
+  }
+
+  /**
+   * Modify an existing cache pool.
+   *
+   * @param info
+   *          The request to modify a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    CachePoolInfo.validate(info);
+    dfs.modifyCachePool(info);
+  }
+    
+  /**
+   * Remove a cache pool.
+   *
+   * @param poolName
+   *          Name of the cache pool to remove.
+   * @throws IOException 
+   *          if the cache pool did not exist, or could not be removed.
+   */
+  public void removeCachePool(String poolName) throws IOException {
+    CachePoolInfo.validateName(poolName);
+    dfs.removeCachePool(poolName);
+  }
+
+  /**
+   * List all cache pools.
+   *
+   * @return A remote iterator from which you can get CachePoolEntry objects.
+   *          Requests will be made as needed.
+   * @throws IOException
+   *          If there was an error listing cache pools.
+   */
+  public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
+    return dfs.listCachePools();
+  }
 }
 }

+ 2 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ClientMmap.java

@@ -22,6 +22,7 @@ import java.io.FileInputStream;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 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.io.nativeio.NativeIO;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.lang.ref.WeakReference;
 import java.lang.ref.WeakReference;
@@ -147,20 +148,9 @@ public class ClientMmap {
 
 
   /**
   /**
    * Unmap the memory region.
    * Unmap the memory region.
-   *
-   * There isn't any portable way to unmap a memory region in Java.
-   * So we use the sun.nio method here.
-   * Note that unmapping a memory region could cause crashes if code
-   * continues to reference the unmapped code.  However, if we don't
-   * manually unmap the memory, we are dependent on the finalizer to
-   * do it, and we have no idea when the finalizer will run.
    */
    */
   void unmap() {
   void unmap() {
     assert(refCount.get() == 0);
     assert(refCount.get() == 0);
-    if (map instanceof sun.nio.ch.DirectBuffer) {
-      final sun.misc.Cleaner cleaner =
-          ((sun.nio.ch.DirectBuffer) map).cleaner();
-      cleaner.clean();
-    }
+    NativeIO.POSIX.munmap(map);
   }
   }
 }
 }

+ 104 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -19,13 +19,20 @@ 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.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 
 
@@ -121,4 +128,101 @@ public class HdfsAdmin {
   public void disallowSnapshot(Path path) throws IOException {
   public void disallowSnapshot(Path path) throws IOException {
     dfs.disallowSnapshot(path);
     dfs.disallowSnapshot(path);
   }
   }
+
+  /**
+   * Add a new CacheDirectiveInfo.
+   * 
+   * @param info Information about a directive to add.
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @return the ID of the directive that was created.
+   * @throws IOException if the directive could not be added
+   */
+  public long addCacheDirective(CacheDirectiveInfo info,
+      EnumSet<CacheFlag> flags) throws IOException {
+  return dfs.addCacheDirective(info, flags);
+  }
+  
+  /**
+   * Modify a CacheDirective.
+   * 
+   * @param info Information about the directive to modify. You must set the ID
+   *          to indicate which CacheDirective you want to modify.
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @throws IOException if the directive could not be modified
+   */
+  public void modifyCacheDirective(CacheDirectiveInfo info,
+      EnumSet<CacheFlag> flags) throws IOException {
+    dfs.modifyCacheDirective(info, flags);
+  }
+
+  /**
+   * Remove a CacheDirective.
+   * 
+   * @param id identifier of the CacheDirectiveInfo to remove
+   * @throws IOException if the directive could not be removed
+   */
+  public void removeCacheDirective(long id)
+      throws IOException {
+    dfs.removeCacheDirective(id);
+  }
+
+  /**
+   * List cache directives. Incrementally fetches results from the server.
+   * 
+   * @param filter Filter parameters to use when listing the directives, null to
+   *               list all directives visible to us.
+   * @return A RemoteIterator which returns CacheDirectiveInfo objects.
+   */
+  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+      CacheDirectiveInfo filter) throws IOException {
+    return dfs.listCacheDirectives(filter);
+  }
+
+  /**
+   * Add a cache pool.
+   *
+   * @param info
+   *          The request to add a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    dfs.addCachePool(info);
+  }
+
+  /**
+   * Modify an existing cache pool.
+   *
+   * @param info
+   *          The request to modify a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    dfs.modifyCachePool(info);
+  }
+    
+  /**
+   * Remove a cache pool.
+   *
+   * @param poolName
+   *          Name of the cache pool to remove.
+   * @throws IOException 
+   *          if the cache pool did not exist, or could not be removed.
+   */
+  public void removeCachePool(String poolName) throws IOException {
+    dfs.removeCachePool(poolName);
+  }
+
+  /**
+   * List all cache pools.
+   *
+   * @return A remote iterator from which you can get CachePoolEntry objects.
+   *          Requests will be made as needed.
+   * @throws IOException
+   *          If there was an error listing cache pools.
+   */
+  public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
+    return dfs.listCachePools();
+  }
 }
 }

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

@@ -0,0 +1,268 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.Date;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.server.namenode.CachePool;
+import org.apache.hadoop.util.IntrusiveCollection;
+import org.apache.hadoop.util.IntrusiveCollection.Element;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Namenode class that tracks state related to a cached path.
+ *
+ * This is an implementation class, not part of the public API.
+ */
+@InterfaceAudience.Private
+public final class CacheDirective implements IntrusiveCollection.Element {
+  private final long id;
+  private final String path;
+  private final short replication;
+  private CachePool pool;
+  private final long expiryTime;
+
+  private long bytesNeeded;
+  private long bytesCached;
+  private long filesNeeded;
+  private long filesCached;
+
+  private Element prev;
+  private Element next;
+
+  public CacheDirective(CacheDirectiveInfo info) {
+    this(
+        info.getId(),
+        info.getPath().toUri().getPath(),
+        info.getReplication(),
+        info.getExpiration().getAbsoluteMillis());
+  }
+
+  public CacheDirective(long id, String path,
+      short replication, long expiryTime) {
+    Preconditions.checkArgument(id > 0);
+    this.id = id;
+    this.path = checkNotNull(path);
+    Preconditions.checkArgument(replication > 0);
+    this.replication = replication;
+    this.expiryTime = expiryTime;
+  }
+
+  public long getId() {
+    return id;
+  }
+
+  public String getPath() {
+    return path;
+  }
+
+  public short getReplication() {
+    return replication;
+  }
+
+  public CachePool getPool() {
+    return pool;
+  }
+
+  /**
+   * @return When this directive expires, in milliseconds since Unix epoch
+   */
+  public long getExpiryTime() {
+    return expiryTime;
+  }
+
+  /**
+   * @return When this directive expires, as an ISO-8601 formatted string.
+   */
+  public String getExpiryTimeString() {
+    return DFSUtil.dateToIso8601String(new Date(expiryTime));
+  }
+
+  /**
+   * Returns a {@link CacheDirectiveInfo} based on this CacheDirective.
+   * <p>
+   * This always sets an absolute expiry time, never a relative TTL.
+   */
+  public CacheDirectiveInfo toInfo() {
+    return new CacheDirectiveInfo.Builder().
+        setId(id).
+        setPath(new Path(path)).
+        setReplication(replication).
+        setPool(pool.getPoolName()).
+        setExpiration(CacheDirectiveInfo.Expiration.newAbsolute(expiryTime)).
+        build();
+  }
+
+  public CacheDirectiveStats toStats() {
+    return new CacheDirectiveStats.Builder().
+        setBytesNeeded(bytesNeeded).
+        setBytesCached(bytesCached).
+        setFilesNeeded(filesNeeded).
+        setFilesCached(filesCached).
+        setHasExpired(new Date().getTime() > expiryTime).
+        build();
+  }
+
+  public CacheDirectiveEntry toEntry() {
+    return new CacheDirectiveEntry(toInfo(), toStats());
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("{ id:").append(id).
+      append(", path:").append(path).
+      append(", replication:").append(replication).
+      append(", pool:").append(pool).
+      append(", expiryTime: ").append(getExpiryTimeString()).
+      append(", bytesNeeded:").append(bytesNeeded).
+      append(", bytesCached:").append(bytesCached).
+      append(", filesNeeded:").append(filesNeeded).
+      append(", filesCached:").append(filesCached).
+      append(" }");
+    return builder.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o == null) { return false; }
+    if (o == this) { return true; }
+    if (o.getClass() != this.getClass()) {
+      return false;
+    }
+    CacheDirective other = (CacheDirective)o;
+    return id == other.id;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().append(id).toHashCode();
+  }
+
+  //
+  // Stats related getters and setters
+  //
+
+  /**
+   * Resets the byte and file statistics being tracked by this CacheDirective.
+   */
+  public void resetStatistics() {
+    bytesNeeded = 0;
+    bytesCached = 0;
+    filesNeeded = 0;
+    filesCached = 0;
+  }
+
+  public long getBytesNeeded() {
+    return bytesNeeded;
+  }
+
+  public void addBytesNeeded(long bytes) {
+    this.bytesNeeded += bytes;
+    pool.addBytesNeeded(bytes);
+  }
+
+  public long getBytesCached() {
+    return bytesCached;
+  }
+
+  public void addBytesCached(long bytes) {
+    this.bytesCached += bytes;
+    pool.addBytesCached(bytes);
+  }
+
+  public long getFilesNeeded() {
+    return filesNeeded;
+  }
+
+  public void addFilesNeeded(long files) {
+    this.filesNeeded += files;
+    pool.addFilesNeeded(files);
+  }
+
+  public long getFilesCached() {
+    return filesCached;
+  }
+
+  public void addFilesCached(long files) {
+    this.filesCached += files;
+    pool.addFilesCached(files);
+  }
+
+  //
+  // IntrusiveCollection.Element implementation
+  //
+
+  @SuppressWarnings("unchecked")
+  @Override // IntrusiveCollection.Element
+  public void insertInternal(IntrusiveCollection<? extends Element> list,
+      Element prev, Element next) {
+    assert this.pool == null;
+    this.pool = ((CachePool.DirectiveList)list).getCachePool();
+    this.prev = prev;
+    this.next = next;
+  }
+
+  @Override // IntrusiveCollection.Element
+  public void setPrev(IntrusiveCollection<? extends Element> list, Element prev) {
+    assert list == pool.getDirectiveList();
+    this.prev = prev;
+  }
+
+  @Override // IntrusiveCollection.Element
+  public void setNext(IntrusiveCollection<? extends Element> list, Element next) {
+    assert list == pool.getDirectiveList();
+    this.next = next;
+  }
+
+  @Override // IntrusiveCollection.Element
+  public void removeInternal(IntrusiveCollection<? extends Element> list) {
+    assert list == pool.getDirectiveList();
+    this.pool = null;
+    this.prev = null;
+    this.next = null;
+  }
+
+  @Override // IntrusiveCollection.Element
+  public Element getPrev(IntrusiveCollection<? extends Element> list) {
+    if (list != pool.getDirectiveList()) {
+      return null;
+    }
+    return this.prev;
+  }
+
+  @Override // IntrusiveCollection.Element
+  public Element getNext(IntrusiveCollection<? extends Element> list) {
+    if (list != pool.getDirectiveList()) {
+      return null;
+    }
+    return this.next;
+  }
+
+  @Override // IntrusiveCollection.Element
+  public boolean isInList(IntrusiveCollection<? extends Element> list) {
+    return pool == null ? false : list == pool.getDirectiveList();
+  }
+};

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java

@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Describes a path-based cache directive entry.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Public
+public class CacheDirectiveEntry {
+  private final CacheDirectiveInfo info;
+  private final CacheDirectiveStats stats;
+
+  public CacheDirectiveEntry(CacheDirectiveInfo info,
+      CacheDirectiveStats stats) {
+    this.info = info;
+    this.stats = stats;
+  }
+
+  public CacheDirectiveInfo getInfo() {
+    return info;
+  }
+
+  public CacheDirectiveStats getStats() {
+    return stats;
+  }
+};

+ 358 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java

@@ -0,0 +1,358 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.util.Date;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Describes a path-based cache directive.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Public
+public class CacheDirectiveInfo {
+  /**
+   * A builder for creating new CacheDirectiveInfo instances.
+   */
+  public static class Builder {
+    private Long id;
+    private Path path;
+    private Short replication;
+    private String pool;
+    private Expiration expiration;
+
+    /**
+     * Builds a new CacheDirectiveInfo populated with the set properties.
+     * 
+     * @return New CacheDirectiveInfo.
+     */
+    public CacheDirectiveInfo build() {
+      return new CacheDirectiveInfo(id, path, replication, pool, expiration);
+    }
+
+    /**
+     * Creates an empty builder.
+     */
+    public Builder() {
+    }
+
+    /**
+     * Creates a builder with all elements set to the same values as the
+     * given CacheDirectiveInfo.
+     */
+    public Builder(CacheDirectiveInfo directive) {
+      this.id = directive.getId();
+      this.path = directive.getPath();
+      this.replication = directive.getReplication();
+      this.pool = directive.getPool();
+      this.expiration = directive.getExpiration();
+    }
+
+    /**
+     * Sets the id used in this request.
+     * 
+     * @param id The id used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setId(Long id) {
+      this.id = id;
+      return this;
+    }
+
+    /**
+     * Sets the path used in this request.
+     * 
+     * @param path The path used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setPath(Path path) {
+      this.path = path;
+      return this;
+    }
+
+    /**
+     * Sets the replication used in this request.
+     * 
+     * @param replication The replication used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setReplication(Short replication) {
+      this.replication = replication;
+      return this;
+    }
+
+    /**
+     * Sets the pool used in this request.
+     * 
+     * @param pool The pool used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setPool(String pool) {
+      this.pool = pool;
+      return this;
+    }
+
+    /**
+     * Sets when the CacheDirective should expire. A
+     * {@link CacheDirectiveInfo.Expiration} can specify either an absolute or
+     * relative expiration time.
+     * 
+     * @param expiration when this CacheDirective should expire
+     * @return This builder, for call chaining
+     */
+    public Builder setExpiration(Expiration expiration) {
+      this.expiration = expiration;
+      return this;
+    }
+  }
+
+  /**
+   * Denotes a relative or absolute expiration time for a CacheDirective. Use
+   * factory methods {@link CacheDirectiveInfo.Expiration#newAbsolute(Date)} and
+   * {@link CacheDirectiveInfo.Expiration#newRelative(long)} to create an
+   * Expiration.
+   * <p>
+   * In either case, the server-side clock is used to determine when a
+   * CacheDirective expires.
+   */
+  public static class Expiration {
+
+    /**
+     * The maximum value we accept for a relative expiry.
+     */
+    public static final long MAX_RELATIVE_EXPIRY_MS =
+        Long.MAX_VALUE / 4; // This helps prevent weird overflow bugs
+
+    /**
+     * An relative Expiration that never expires.
+     */
+    public static final Expiration NEVER = newRelative(MAX_RELATIVE_EXPIRY_MS);
+
+    /**
+     * Create a new relative Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
+     * 
+     * @param ms how long until the CacheDirective expires, in milliseconds
+     * @return A relative Expiration
+     */
+    public static Expiration newRelative(long ms) {
+      return new Expiration(ms, true);
+    }
+
+    /**
+     * Create a new absolute Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
+     * 
+     * @param date when the CacheDirective expires
+     * @return An absolute Expiration
+     */
+    public static Expiration newAbsolute(Date date) {
+      return new Expiration(date.getTime(), false);
+    }
+
+    /**
+     * Create a new absolute Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
+     * 
+     * @param ms when the CacheDirective expires, in milliseconds since the Unix
+     *          epoch.
+     * @return An absolute Expiration
+     */
+    public static Expiration newAbsolute(long ms) {
+      return new Expiration(ms, false);
+    }
+
+    private final long ms;
+    private final boolean isRelative;
+
+    private Expiration(long ms, boolean isRelative) {
+      if (isRelative) {
+        Preconditions.checkArgument(ms <= MAX_RELATIVE_EXPIRY_MS,
+            "Expiration time is too far in the future!");
+      }
+      this.ms = ms;
+      this.isRelative = isRelative;
+    }
+
+    /**
+     * @return true if Expiration was specified as a relative duration, false if
+     *         specified as an absolute time.
+     */
+    public boolean isRelative() {
+      return isRelative;
+    }
+
+    /**
+     * @return The raw underlying millisecond value, either a relative duration
+     *         or an absolute time as milliseconds since the Unix epoch.
+     */
+    public long getMillis() {
+      return ms;
+    }
+
+    /**
+     * @return Expiration time as a {@link Date} object. This converts a
+     *         relative Expiration into an absolute Date based on the local
+     *         clock.
+     */
+    public Date getAbsoluteDate() {
+      return new Date(getAbsoluteMillis());
+    }
+
+    /**
+     * @return Expiration time in milliseconds from the Unix epoch. This
+     *         converts a relative Expiration into an absolute time based on the
+     *         local clock.
+     */
+    public long getAbsoluteMillis() {
+      if (!isRelative) {
+        return ms;
+      } else {
+        return new Date().getTime() + ms;
+      }
+    }
+
+    @Override
+    public String toString() {
+      if (isRelative) {
+        return DFSUtil.durationToString(ms);
+      }
+      return DFSUtil.dateToIso8601String(new Date(ms));
+    }
+  }
+
+  private final Long id;
+  private final Path path;
+  private final Short replication;
+  private final String pool;
+  private final Expiration expiration;
+
+  CacheDirectiveInfo(Long id, Path path, Short replication, String pool,
+      Expiration expiration) {
+    this.id = id;
+    this.path = path;
+    this.replication = replication;
+    this.pool = pool;
+    this.expiration = expiration;
+  }
+
+  /**
+   * @return The ID of this directive.
+   */
+  public Long getId() {
+    return id;
+  }
+
+  /**
+   * @return The path used in this request.
+   */
+  public Path getPath() {
+    return path;
+  }
+
+  /**
+   * @return The number of times the block should be cached.
+   */
+  public Short getReplication() {
+    return replication;
+  }
+
+  /**
+   * @return The pool used in this request.
+   */
+  public String getPool() {
+    return pool;
+  }
+
+  /**
+   * @return When this directive expires.
+   */
+  public Expiration getExpiration() {
+    return expiration;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o == null) {
+      return false;
+    }
+    if (getClass() != o.getClass()) {
+      return false;
+    }
+    CacheDirectiveInfo other = (CacheDirectiveInfo)o;
+    return new EqualsBuilder().append(getId(), other.getId()).
+        append(getPath(), other.getPath()).
+        append(getReplication(), other.getReplication()).
+        append(getPool(), other.getPool()).
+        append(getExpiration(), other.getExpiration()).
+        isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().append(id).
+        append(path).
+        append(replication).
+        append(pool).
+        append(expiration).
+        hashCode();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("{");
+    String prefix = "";
+    if (id != null) {
+      builder.append(prefix).append("id: ").append(id);
+      prefix = ", ";
+    }
+    if (path != null) {
+      builder.append(prefix).append("path: ").append(path);
+      prefix = ", ";
+    }
+    if (replication != null) {
+      builder.append(prefix).append("replication: ").append(replication);
+      prefix = ", ";
+    }
+    if (pool != null) {
+      builder.append(prefix).append("pool: ").append(pool);
+      prefix = ", ";
+    }
+    if (expiration != null) {
+      builder.append(prefix).append("expiration: ").append(expiration);
+      prefix = ", ";
+    }
+    builder.append("}");
+    return builder.toString();
+  }
+};

+ 56 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java

@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+
+/**
+ * CacheDirectiveIterator is a remote iterator that iterates cache directives.
+ * It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CacheDirectiveIterator
+    extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
+
+  private final CacheDirectiveInfo filter;
+  private final ClientProtocol namenode;
+
+  public CacheDirectiveIterator(ClientProtocol namenode,
+      CacheDirectiveInfo filter) {
+    super(Long.valueOf(0));
+    this.namenode = namenode;
+    this.filter = filter;
+  }
+
+  @Override
+  public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
+      throws IOException {
+    return namenode.listCacheDirectives(prevKey, filter);
+  }
+
+  @Override
+  public Long elementToPrevKey(CacheDirectiveEntry entry) {
+    return entry.getInfo().getId();
+  }
+}

+ 169 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java

@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Describes a path-based cache directive.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Public
+public class CacheDirectiveStats {
+  public static class Builder {
+    private long bytesNeeded;
+    private long bytesCached;
+    private long filesNeeded;
+    private long filesCached;
+    private boolean hasExpired;
+
+    /**
+     * Builds a new CacheDirectiveStats populated with the set properties.
+     * 
+     * @return New CacheDirectiveStats.
+     */
+    public CacheDirectiveStats build() {
+      return new CacheDirectiveStats(bytesNeeded, bytesCached, filesNeeded,
+          filesCached, hasExpired);
+    }
+
+    /**
+     * Creates an empty builder.
+     */
+    public Builder() {
+    }
+
+    /**
+     * Sets the bytes needed by this directive.
+     * 
+     * @param bytesNeeded The bytes needed.
+     * @return This builder, for call chaining.
+     */
+    public Builder setBytesNeeded(long bytesNeeded) {
+      this.bytesNeeded = bytesNeeded;
+      return this;
+    }
+
+    /**
+     * Sets the bytes cached by this directive.
+     * 
+     * @param bytesCached The bytes cached.
+     * @return This builder, for call chaining.
+     */
+    public Builder setBytesCached(long bytesCached) {
+      this.bytesCached = bytesCached;
+      return this;
+    }
+
+    /**
+     * Sets the files needed by this directive.
+     * @param filesNeeded The number of files needed
+     * @return This builder, for call chaining.
+     */
+    public Builder setFilesNeeded(long filesNeeded) {
+      this.filesNeeded = filesNeeded;
+      return this;
+    }
+
+    /**
+     * Sets the files cached by this directive.
+     * 
+     * @param filesCached The number of files cached.
+     * @return This builder, for call chaining.
+     */
+    public Builder setFilesCached(long filesCached) {
+      this.filesCached = filesCached;
+      return this;
+    }
+
+    /**
+     * Sets whether this directive has expired.
+     * 
+     * @param hasExpired if this directive has expired
+     * @return This builder, for call chaining.
+     */
+    public Builder setHasExpired(boolean hasExpired) {
+      this.hasExpired = hasExpired;
+      return this;
+    }
+  }
+
+  private final long bytesNeeded;
+  private final long bytesCached;
+  private final long filesNeeded;
+  private final long filesCached;
+  private final boolean hasExpired;
+
+  private CacheDirectiveStats(long bytesNeeded, long bytesCached,
+      long filesNeeded, long filesCached, boolean hasExpired) {
+    this.bytesNeeded = bytesNeeded;
+    this.bytesCached = bytesCached;
+    this.filesNeeded = filesNeeded;
+    this.filesCached = filesCached;
+    this.hasExpired = hasExpired;
+  }
+
+  /**
+   * @return The bytes needed.
+   */
+  public long getBytesNeeded() {
+    return bytesNeeded;
+  }
+
+  /**
+   * @return The bytes cached.
+   */
+  public long getBytesCached() {
+    return bytesCached;
+  }
+
+  /**
+   * @return The number of files needed.
+   */
+  public long getFilesNeeded() {
+    return filesNeeded;
+  }
+
+  /**
+   * @return The number of files cached.
+   */
+  public long getFilesCached() {
+    return filesCached;
+  }
+
+  /**
+   * @return Whether this directive has expired.
+   */
+  public boolean hasExpired() {
+    return hasExpired;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("{");
+    builder.append("bytesNeeded: ").append(bytesNeeded);
+    builder.append(", ").append("bytesCached: ").append(bytesCached);
+    builder.append(", ").append("filesNeeded: ").append(filesNeeded);
+    builder.append(", ").append("filesCached: ").append(filesCached);
+    builder.append(", ").append("hasExpired: ").append(hasExpired);
+    builder.append("}");
+    return builder.toString();
+  }
+};

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java

@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Describes a Cache Pool entry.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CachePoolEntry {
+  private final CachePoolInfo info;
+  private final CachePoolStats stats;
+
+  public CachePoolEntry(CachePoolInfo info, CachePoolStats stats) {
+    this.info = info;
+    this.stats = stats;
+  }
+
+  public CachePoolInfo getInfo() {
+    return info;
+  }
+
+  public CachePoolStats getStats() {
+    return stats;
+  }
+}

+ 229 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java

@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import javax.annotation.Nullable;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
+
+/**
+ * CachePoolInfo describes a cache pool.
+ *
+ * This class is used in RPCs to create and modify cache pools.
+ * It is serializable and can be stored in the edit log.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CachePoolInfo {
+  public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
+
+  /**
+   * Indicates that the pool does not have a maximum relative expiry.
+   */
+  public static final long RELATIVE_EXPIRY_NEVER =
+      Expiration.MAX_RELATIVE_EXPIRY_MS;
+  /**
+   * Default max relative expiry for cache pools.
+   */
+  public static final long DEFAULT_MAX_RELATIVE_EXPIRY =
+      RELATIVE_EXPIRY_NEVER;
+
+  public static final long LIMIT_UNLIMITED = Long.MAX_VALUE;
+  public static final long DEFAULT_LIMIT = LIMIT_UNLIMITED;
+
+  final String poolName;
+
+  @Nullable
+  String ownerName;
+
+  @Nullable
+  String groupName;
+
+  @Nullable
+  FsPermission mode;
+
+  @Nullable
+  Long limit;
+
+  @Nullable
+  Long maxRelativeExpiryMs;
+
+  public CachePoolInfo(String poolName) {
+    this.poolName = poolName;
+  }
+
+  /**
+   * @return Name of the pool.
+   */
+  public String getPoolName() {
+    return poolName;
+  }
+
+  /**
+   * @return The owner of the pool. Along with the group and mode, determines
+   *         who has access to view and modify the pool.
+   */
+  public String getOwnerName() {
+    return ownerName;
+  }
+
+  public CachePoolInfo setOwnerName(String ownerName) {
+    this.ownerName = ownerName;
+    return this;
+  }
+
+  /**
+   * @return The group of the pool. Along with the owner and mode, determines
+   *         who has access to view and modify the pool.
+   */
+  public String getGroupName() {
+    return groupName;
+  }
+
+  public CachePoolInfo setGroupName(String groupName) {
+    this.groupName = groupName;
+    return this;
+  }
+
+  /**
+   * @return Unix-style permissions of the pool. Along with the owner and group,
+   *         determines who has access to view and modify the pool.
+   */
+  public FsPermission getMode() {
+    return mode;
+  }
+
+  public CachePoolInfo setMode(FsPermission mode) {
+    this.mode = mode;
+    return this;
+  }
+
+  /**
+   * @return The maximum aggregate number of bytes that can be cached by
+   *         directives in this pool.
+   */
+  public Long getLimit() {
+    return limit;
+  }
+
+  public CachePoolInfo setLimit(Long bytes) {
+    this.limit = bytes;
+    return this;
+  }
+
+  /**
+   * @return The maximum relative expiration of directives of this pool in
+   *         milliseconds
+   */
+  public Long getMaxRelativeExpiryMs() {
+    return maxRelativeExpiryMs;
+  }
+
+  /**
+   * Set the maximum relative expiration of directives of this pool in
+   * milliseconds.
+   * 
+   * @param ms in milliseconds
+   * @return This builder, for call chaining.
+   */
+  public CachePoolInfo setMaxRelativeExpiryMs(Long ms) {
+    this.maxRelativeExpiryMs = ms;
+    return this;
+  }
+
+  public String toString() {
+    return new StringBuilder().append("{").
+      append("poolName:").append(poolName).
+      append(", ownerName:").append(ownerName).
+      append(", groupName:").append(groupName).
+      append(", mode:").append((mode == null) ? "null" :
+          String.format("0%03o", mode.toShort())).
+      append(", limit:").append(limit).
+      append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
+      append("}").toString();
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (o == null) { return false; }
+    if (o == this) { return true; }
+    if (o.getClass() != getClass()) {
+      return false;
+    }
+    CachePoolInfo other = (CachePoolInfo)o;
+    return new EqualsBuilder().
+        append(poolName, other.poolName).
+        append(ownerName, other.ownerName).
+        append(groupName, other.groupName).
+        append(mode, other.mode).
+        append(limit, other.limit).
+        append(maxRelativeExpiryMs, other.maxRelativeExpiryMs).
+        isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().
+        append(poolName).
+        append(ownerName).
+        append(groupName).
+        append(mode).
+        append(limit).
+        append(maxRelativeExpiryMs).
+        hashCode();
+  }
+
+  public static void validate(CachePoolInfo info) throws IOException {
+    if (info == null) {
+      throw new InvalidRequestException("CachePoolInfo is null");
+    }
+    if ((info.getLimit() != null) && (info.getLimit() < 0)) {
+      throw new InvalidRequestException("Limit is negative.");
+    }
+    if (info.getMaxRelativeExpiryMs() != null) {
+      long maxRelativeExpiryMs = info.getMaxRelativeExpiryMs();
+      if (maxRelativeExpiryMs < 0l) {
+        throw new InvalidRequestException("Max relative expiry is negative.");
+      }
+      if (maxRelativeExpiryMs > Expiration.MAX_RELATIVE_EXPIRY_MS) {
+        throw new InvalidRequestException("Max relative expiry is too big.");
+      }
+    }
+    validateName(info.poolName);
+  }
+
+  public static void validateName(String poolName) throws IOException {
+    if (poolName == null || poolName.isEmpty()) {
+      // Empty pool names are not allowed because they would be highly
+      // confusing.  They would also break the ability to list all pools
+      // by starting with prevKey = ""
+      throw new IOException("invalid empty cache pool name");
+    }
+  }
+}

+ 53 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+
+/**
+ * CachePoolIterator is a remote iterator that iterates cache pools.
+ * It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CachePoolIterator
+    extends BatchedRemoteIterator<String, CachePoolEntry> {
+
+  private final ClientProtocol namenode;
+
+  public CachePoolIterator(ClientProtocol namenode) {
+    super("");
+    this.namenode = namenode;
+  }
+
+  @Override
+  public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
+      throws IOException {
+    return namenode.listCachePools(prevKey);
+  }
+
+  @Override
+  public String elementToPrevKey(CachePoolEntry entry) {
+    return entry.getInfo().getPoolName();
+  }
+}

+ 115 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java

@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * CachePoolStats describes cache pool statistics.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CachePoolStats {
+  public static class Builder {
+    private long bytesNeeded;
+    private long bytesCached;
+    private long bytesOverlimit;
+    private long filesNeeded;
+    private long filesCached;
+
+    public Builder() {
+    }
+
+    public Builder setBytesNeeded(long bytesNeeded) {
+      this.bytesNeeded = bytesNeeded;
+      return this;
+    }
+
+    public Builder setBytesCached(long bytesCached) {
+      this.bytesCached = bytesCached;
+      return this;
+    }
+
+    public Builder setBytesOverlimit(long bytesOverlimit) {
+      this.bytesOverlimit = bytesOverlimit;
+      return this;
+    }
+
+    public Builder setFilesNeeded(long filesNeeded) {
+      this.filesNeeded = filesNeeded;
+      return this;
+    }
+
+    public Builder setFilesCached(long filesCached) {
+      this.filesCached = filesCached;
+      return this;
+    }
+
+    public CachePoolStats build() {
+      return new CachePoolStats(bytesNeeded, bytesCached, bytesOverlimit,
+          filesNeeded, filesCached);
+    }
+  };
+
+  private final long bytesNeeded;
+  private final long bytesCached;
+  private final long bytesOverlimit;
+  private final long filesNeeded;
+  private final long filesCached;
+
+  private CachePoolStats(long bytesNeeded, long bytesCached,
+      long bytesOverlimit, long filesNeeded, long filesCached) {
+    this.bytesNeeded = bytesNeeded;
+    this.bytesCached = bytesCached;
+    this.bytesOverlimit = bytesOverlimit;
+    this.filesNeeded = filesNeeded;
+    this.filesCached = filesCached;
+  }
+
+  public long getBytesNeeded() {
+    return bytesNeeded;
+  }
+
+  public long getBytesCached() {
+    return bytesCached;
+  }
+
+  public long getBytesOverlimit() {
+    return bytesOverlimit;
+  }
+
+  public long getFilesNeeded() {
+    return filesNeeded;
+  }
+
+  public long getFilesCached() {
+    return filesCached;
+  }
+
+  public String toString() {
+    return new StringBuilder().append("{").
+      append("bytesNeeded:").append(bytesNeeded).
+      append(", bytesCached:").append(bytesCached).
+      append(", bytesOverlimit:").append(bytesOverlimit).
+      append(", filesNeeded:").append(filesNeeded).
+      append(", filesCached:").append(filesCached).
+      append("}").toString();
+  }
+}

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

@@ -19,15 +19,18 @@ 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 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;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
@@ -1094,5 +1097,91 @@ public interface ClientProtocol {
   @Idempotent
   @Idempotent
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String fromSnapshot, String toSnapshot) throws IOException;
       String fromSnapshot, String toSnapshot) throws IOException;
-}
 
 
+  /**
+   * Add a CacheDirective to the CacheManager.
+   * 
+   * @param directive A CacheDirectiveInfo to be added
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @return A CacheDirectiveInfo associated with the added directive
+   * @throws IOException if the directive could not be added
+   */
+  @AtMostOnce
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
+
+  /**
+   * Modify a CacheDirective in the CacheManager.
+   * 
+   * @return directive The directive to modify. Must contain a directive ID.
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @throws IOException if the directive could not be modified
+   */
+  @AtMostOnce
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
+
+  /**
+   * Remove a CacheDirectiveInfo from the CacheManager.
+   * 
+   * @param id of a CacheDirectiveInfo
+   * @throws IOException if the cache directive could not be removed
+   */
+  @AtMostOnce
+  public void removeCacheDirective(long id) throws IOException;
+
+  /**
+   * List the set of cached paths of a cache pool. Incrementally fetches results
+   * from the server.
+   * 
+   * @param prevId The last listed entry ID, or -1 if this is the first call to
+   *               listCacheDirectives.
+   * @param filter Parameters to use to filter the list results, 
+   *               or null to display all directives visible to us.
+   * @return A batch of CacheDirectiveEntry objects.
+   */
+  @Idempotent
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
+      long prevId, CacheDirectiveInfo filter) throws IOException;
+
+  /**
+   * Add a new cache pool.
+   * 
+   * @param info Description of the new cache pool
+   * @throws IOException If the request could not be completed.
+   */
+  @AtMostOnce
+  public void addCachePool(CachePoolInfo info) throws IOException;
+
+  /**
+   * Modify an existing cache pool.
+   *
+   * @param req
+   *          The request to modify a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  @AtMostOnce
+  public void modifyCachePool(CachePoolInfo req) throws IOException;
+  
+  /**
+   * Remove a cache pool.
+   * 
+   * @param pool name of the cache pool to remove.
+   * @throws IOException if the cache pool did not exist, or could not be
+   *           removed.
+   */
+  @AtMostOnce
+  public void removeCachePool(String pool) throws IOException;
+
+  /**
+   * List the set of cache pools. Incrementally fetches results from the server.
+   * 
+   * @param prevPool name of the last pool listed, or the empty string if this is
+   *          the first invocation of listCachePools
+   * @return A batch of CachePoolEntry objects.
+   */
+  @Idempotent
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
+      throws IOException;
+}

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

@@ -44,6 +44,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
   private long dfsUsed;
   private long dfsUsed;
   private long remaining;
   private long remaining;
   private long blockPoolUsed;
   private long blockPoolUsed;
+  private long cacheCapacity;
+  private long cacheUsed;
   private long lastUpdate;
   private long lastUpdate;
   private int xceiverCount;
   private int xceiverCount;
   private String location = NetworkTopology.DEFAULT_RACK;
   private String location = NetworkTopology.DEFAULT_RACK;
@@ -82,6 +84,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.dfsUsed = from.getDfsUsed();
     this.dfsUsed = from.getDfsUsed();
     this.remaining = from.getRemaining();
     this.remaining = from.getRemaining();
     this.blockPoolUsed = from.getBlockPoolUsed();
     this.blockPoolUsed = from.getBlockPoolUsed();
+    this.cacheCapacity = from.getCacheCapacity();
+    this.cacheUsed = from.getCacheUsed();
     this.lastUpdate = from.getLastUpdate();
     this.lastUpdate = from.getLastUpdate();
     this.xceiverCount = from.getXceiverCount();
     this.xceiverCount = from.getXceiverCount();
     this.location = from.getNetworkLocation();
     this.location = from.getNetworkLocation();
@@ -94,6 +98,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.dfsUsed = 0L;
     this.dfsUsed = 0L;
     this.remaining = 0L;
     this.remaining = 0L;
     this.blockPoolUsed = 0L;
     this.blockPoolUsed = 0L;
+    this.cacheCapacity = 0L;
+    this.cacheUsed = 0L;
     this.lastUpdate = 0L;
     this.lastUpdate = 0L;
     this.xceiverCount = 0;
     this.xceiverCount = 0;
     this.adminState = null;    
     this.adminState = null;    
@@ -106,12 +112,13 @@ public class DatanodeInfo extends DatanodeID implements Node {
   
   
   public DatanodeInfo(DatanodeID nodeID, String location,
   public DatanodeInfo(DatanodeID nodeID, String location,
       final long capacity, final long dfsUsed, final long remaining,
       final long capacity, final long dfsUsed, final long remaining,
-      final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
+      final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
+      final long lastUpdate, final int xceiverCount,
       final AdminStates adminState) {
       final AdminStates adminState) {
     this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getDatanodeUuid(),
     this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getDatanodeUuid(),
         nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
         nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
         nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
         nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
-        lastUpdate, xceiverCount, location, adminState);
+        cacheCapacity, cacheUsed, lastUpdate, xceiverCount, location, adminState);
   }
   }
 
 
   /** Constructor */
   /** Constructor */
@@ -119,7 +126,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
       final String datanodeUuid, final int xferPort, final int infoPort,
       final String datanodeUuid, final int xferPort, final int infoPort,
       final int infoSecurePort, final int ipcPort,
       final int infoSecurePort, final int ipcPort,
       final long capacity, final long dfsUsed, final long remaining,
       final long capacity, final long dfsUsed, final long remaining,
-      final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
+      final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
+      final long lastUpdate, final int xceiverCount,
       final String networkLocation, final AdminStates adminState) {
       final String networkLocation, final AdminStates adminState) {
     super(ipAddr, hostName, datanodeUuid, xferPort, infoPort,
     super(ipAddr, hostName, datanodeUuid, xferPort, infoPort,
             infoSecurePort, ipcPort);
             infoSecurePort, ipcPort);
@@ -127,6 +135,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.dfsUsed = dfsUsed;
     this.dfsUsed = dfsUsed;
     this.remaining = remaining;
     this.remaining = remaining;
     this.blockPoolUsed = blockPoolUsed;
     this.blockPoolUsed = blockPoolUsed;
+    this.cacheCapacity = cacheCapacity;
+    this.cacheUsed = cacheUsed;
     this.lastUpdate = lastUpdate;
     this.lastUpdate = lastUpdate;
     this.xceiverCount = xceiverCount;
     this.xceiverCount = xceiverCount;
     this.location = networkLocation;
     this.location = networkLocation;
@@ -172,6 +182,42 @@ public class DatanodeInfo extends DatanodeID implements Node {
     return DFSUtil.getPercentRemaining(remaining, capacity);
     return DFSUtil.getPercentRemaining(remaining, capacity);
   }
   }
 
 
+  /**
+   * @return Amount of cache capacity in bytes
+   */
+  public long getCacheCapacity() {
+    return cacheCapacity;
+  }
+
+  /**
+   * @return Amount of cache used in bytes
+   */
+  public long getCacheUsed() {
+    return cacheUsed;
+  }
+
+  /**
+   * @return Cache used as a percentage of the datanode's total cache capacity
+   */
+  public float getCacheUsedPercent() {
+    return DFSUtil.getPercentUsed(cacheUsed, cacheCapacity);
+  }
+
+  /**
+   * @return Amount of cache remaining in bytes
+   */
+  public long getCacheRemaining() {
+    return cacheCapacity - cacheUsed;
+  }
+
+  /**
+   * @return Cache remaining as a percentage of the datanode's total cache
+   * capacity
+   */
+  public float getCacheRemainingPercent() {
+    return DFSUtil.getPercentRemaining(getCacheRemaining(), cacheCapacity);
+  }
+
   /** The time when this information was accurate. */
   /** The time when this information was accurate. */
   public long getLastUpdate() { return lastUpdate; }
   public long getLastUpdate() { return lastUpdate; }
 
 
@@ -198,6 +244,16 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.blockPoolUsed = bpUsed; 
     this.blockPoolUsed = bpUsed; 
   }
   }
 
 
+  /** Sets cache capacity. */
+  public void setCacheCapacity(long cacheCapacity) {
+    this.cacheCapacity = cacheCapacity;
+  }
+
+  /** Sets cache used. */
+  public void setCacheUsed(long cacheUsed) {
+    this.cacheUsed = cacheUsed;
+  }
+
   /** Sets time when this information was accurate. */
   /** Sets time when this information was accurate. */
   public void setLastUpdate(long lastUpdate) { 
   public void setLastUpdate(long lastUpdate) { 
     this.lastUpdate = lastUpdate; 
     this.lastUpdate = lastUpdate; 
@@ -225,6 +281,11 @@ public class DatanodeInfo extends DatanodeID implements Node {
     long nonDFSUsed = getNonDfsUsed();
     long nonDFSUsed = getNonDfsUsed();
     float usedPercent = getDfsUsedPercent();
     float usedPercent = getDfsUsedPercent();
     float remainingPercent = getRemainingPercent();
     float remainingPercent = getRemainingPercent();
+    long cc = getCacheCapacity();
+    long cr = getCacheRemaining();
+    long cu = getCacheUsed();
+    float cacheUsedPercent = getCacheUsedPercent();
+    float cacheRemainingPercent = getCacheRemainingPercent();
     String lookupName = NetUtils.getHostNameOfIP(getName());
     String lookupName = NetUtils.getHostNameOfIP(getName());
 
 
     buffer.append("Name: "+ getName());
     buffer.append("Name: "+ getName());
@@ -251,6 +312,12 @@ public class DatanodeInfo extends DatanodeID implements Node {
     buffer.append("DFS Remaining: " +r+ " ("+StringUtils.byteDesc(r)+")"+"\n");
     buffer.append("DFS Remaining: " +r+ " ("+StringUtils.byteDesc(r)+")"+"\n");
     buffer.append("DFS Used%: "+percent2String(usedPercent) + "\n");
     buffer.append("DFS Used%: "+percent2String(usedPercent) + "\n");
     buffer.append("DFS Remaining%: "+percent2String(remainingPercent) + "\n");
     buffer.append("DFS Remaining%: "+percent2String(remainingPercent) + "\n");
+    buffer.append("Configured Cache Capacity: "+cc+" ("+StringUtils.byteDesc(cc)+")"+"\n");
+    buffer.append("Cache Used: "+cu+" ("+StringUtils.byteDesc(cu)+")"+"\n");
+    buffer.append("Cache Remaining: " +cr+ " ("+StringUtils.byteDesc(cr)+")"+"\n");
+    buffer.append("Cache Used%: "+percent2String(cacheUsedPercent) + "\n");
+    buffer.append("Cache Remaining%: "+percent2String(cacheRemainingPercent) + "\n");
+
     buffer.append("Last contact: "+new Date(lastUpdate)+"\n");
     buffer.append("Last contact: "+new Date(lastUpdate)+"\n");
     return buffer.toString();
     return buffer.toString();
   }
   }
@@ -261,6 +328,9 @@ public class DatanodeInfo extends DatanodeID implements Node {
     long c = getCapacity();
     long c = getCapacity();
     long r = getRemaining();
     long r = getRemaining();
     long u = getDfsUsed();
     long u = getDfsUsed();
+    long cc = getCacheCapacity();
+    long cr = getCacheRemaining();
+    long cu = getCacheUsed();
     buffer.append(getName());
     buffer.append(getName());
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
       buffer.append(" "+location);
       buffer.append(" "+location);
@@ -276,6 +346,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
     buffer.append(" " + u + "(" + StringUtils.byteDesc(u)+")");
     buffer.append(" " + u + "(" + StringUtils.byteDesc(u)+")");
     buffer.append(" " + percent2String(u/(double)c));
     buffer.append(" " + percent2String(u/(double)c));
     buffer.append(" " + r + "(" + StringUtils.byteDesc(r)+")");
     buffer.append(" " + r + "(" + StringUtils.byteDesc(r)+")");
+    buffer.append(" " + cc + "(" + StringUtils.byteDesc(cc)+")");
+    buffer.append(" " + cu + "(" + StringUtils.byteDesc(cu)+")");
+    buffer.append(" " + percent2String(cu/(double)cc));
+    buffer.append(" " + cr + "(" + StringUtils.byteDesc(cr)+")");
     buffer.append(" " + new Date(lastUpdate));
     buffer.append(" " + new Date(lastUpdate));
     return buffer.toString();
     return buffer.toString();
   }
   }

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

@@ -111,7 +111,8 @@ public class LayoutVersion {
         + "the new block instead of the entire block list"),
         + "the new block instead of the entire block list"),
     ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
     ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
         + " Use distinct StorageUuid per storage directory."),
         + " Use distinct StorageUuid per storage directory."),
-    ADD_LAYOUT_FLAGS(-50, "Add support for layout flags.");
+    ADD_LAYOUT_FLAGS(-50, "Add support for layout flags."),
+    CACHING(-51, "Support for cache pools and path-based caching");
 
 
     final int lv;
     final int lv;
     final int ancestorLV;
     final int ancestorLV;

+ 56 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -17,6 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.protocol;
 package org.apache.hadoop.hdfs.protocol;
 
 
+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.hdfs.StorageType;
 import org.apache.hadoop.hdfs.StorageType;
@@ -24,10 +26,14 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
 /**
 /**
  * Associates a block with the Datanodes that contain its replicas
  * Associates a block with the Datanodes that contain its replicas
  * and other block metadata (E.g. the file offset associated with this
  * and other block metadata (E.g. the file offset associated with this
- * block, whether it is corrupt, security token, etc).
+ * block, whether it is corrupt, a location is cached in memory,
+ * security token, etc).
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
@@ -45,6 +51,13 @@ public class LocatedBlock {
   // their locations are not part of this object
   // their locations are not part of this object
   private boolean corrupt;
   private boolean corrupt;
   private Token<BlockTokenIdentifier> blockToken = new Token<BlockTokenIdentifier>();
   private Token<BlockTokenIdentifier> blockToken = new Token<BlockTokenIdentifier>();
+  /**
+   * List of cached datanode locations
+   */
+  private DatanodeInfo[] cachedLocs;
+
+  // Used when there are no locations
+  private static final DatanodeInfo[] EMPTY_LOCS = new DatanodeInfo[0];
 
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
     this(b, locs, -1, false); // startOffset is unknown
     this(b, locs, -1, false); // startOffset is unknown
@@ -52,7 +65,7 @@ public class LocatedBlock {
 
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset, 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset, 
                       boolean corrupt) {
                       boolean corrupt) {
-    this(b, locs, null, null, startOffset, corrupt);
+    this(b, locs, null, null, startOffset, corrupt, EMPTY_LOCS);
   }
   }
 
 
   public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages) {
   public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages) {
@@ -61,7 +74,7 @@ public class LocatedBlock {
 
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
                       String[] storageIDs, StorageType[] storageTypes) {
                       String[] storageIDs, StorageType[] storageTypes) {
-    this(b, locs, storageIDs, storageTypes, -1, false);
+    this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
   }
   }
 
 
   public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
   public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
@@ -69,22 +82,29 @@ public class LocatedBlock {
     this(b, DatanodeStorageInfo.toDatanodeInfos(storages),
     this(b, DatanodeStorageInfo.toDatanodeInfos(storages),
         DatanodeStorageInfo.toStorageIDs(storages),
         DatanodeStorageInfo.toStorageIDs(storages),
         DatanodeStorageInfo.toStorageTypes(storages),
         DatanodeStorageInfo.toStorageTypes(storages),
-        startOffset, corrupt); // startOffset is unknown
+        startOffset, corrupt, EMPTY_LOCS); // startOffset is unknown
   }
   }
 
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
                       StorageType[] storageTypes, long startOffset,
                       StorageType[] storageTypes, long startOffset,
-                      boolean corrupt) {
+                      boolean corrupt, DatanodeInfo[] cachedLocs) {
     this.b = b;
     this.b = b;
     this.offset = startOffset;
     this.offset = startOffset;
     this.corrupt = corrupt;
     this.corrupt = corrupt;
     if (locs==null) {
     if (locs==null) {
-      this.locs = new DatanodeInfo[0];
+      this.locs = EMPTY_LOCS;
     } else {
     } else {
       this.locs = locs;
       this.locs = locs;
     }
     }
     this.storageIDs = storageIDs;
     this.storageIDs = storageIDs;
     this.storageTypes = storageTypes;
     this.storageTypes = storageTypes;
+    Preconditions.checkArgument(cachedLocs != null,
+        "cachedLocs should not be null, use a different constructor");
+    if (cachedLocs.length == 0) {
+      this.cachedLocs = EMPTY_LOCS;
+    } else {
+      this.cachedLocs = cachedLocs;
+    }
   }
   }
 
 
   public Token<BlockTokenIdentifier> getBlockToken() {
   public Token<BlockTokenIdentifier> getBlockToken() {
@@ -131,6 +151,36 @@ public class LocatedBlock {
     return this.corrupt;
     return this.corrupt;
   }
   }
 
 
+  /**
+   * Add a the location of a cached replica of the block.
+   * 
+   * @param loc of datanode with the cached replica
+   */
+  public void addCachedLoc(DatanodeInfo loc) {
+    List<DatanodeInfo> cachedList = Lists.newArrayList(cachedLocs);
+    if (cachedList.contains(loc)) {
+      return;
+    }
+    // Try to re-use a DatanodeInfo already in loc
+    for (int i=0; i<locs.length; i++) {
+      if (locs[i].equals(loc)) {
+        cachedList.add(locs[i]);
+        cachedLocs = cachedList.toArray(cachedLocs);
+        return;
+      }
+    }
+    // Not present in loc, add it and go
+    cachedList.add(loc);
+    cachedLocs = cachedList.toArray(cachedLocs);
+  }
+
+  /**
+   * @return Datanodes with a cached block replica
+   */
+  public DatanodeInfo[] getCachedLocations() {
+    return cachedLocs;
+  }
+
   @Override
   @Override
   public String toString() {
   public String toString() {
     return getClass().getSimpleName() + "{" + b
     return getClass().getSimpleName() + "{" + b

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

@@ -24,19 +24,27 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
@@ -92,16 +100,28 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
@@ -300,7 +320,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) {
@@ -1008,5 +1028,119 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
       throw new ServiceException(e);
     }
     }
   }
   }
+
+  @Override
+  public AddCacheDirectiveResponseProto addCacheDirective(
+      RpcController controller, AddCacheDirectiveRequestProto request)
+      throws ServiceException {
+    try {
+      long id = server.addCacheDirective(
+          PBHelper.convert(request.getInfo()),
+          PBHelper.convertCacheFlags(request.getCacheFlags()));
+      return AddCacheDirectiveResponseProto.newBuilder().
+              setId(id).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ModifyCacheDirectiveResponseProto modifyCacheDirective(
+      RpcController controller, ModifyCacheDirectiveRequestProto request)
+      throws ServiceException {
+    try {
+      server.modifyCacheDirective(
+          PBHelper.convert(request.getInfo()),
+          PBHelper.convertCacheFlags(request.getCacheFlags()));
+      return ModifyCacheDirectiveResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RemoveCacheDirectiveResponseProto
+      removeCacheDirective(RpcController controller,
+          RemoveCacheDirectiveRequestProto request)
+              throws ServiceException {
+    try {
+      server.removeCacheDirective(request.getId());
+      return RemoveCacheDirectiveResponseProto.
+          newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ListCacheDirectivesResponseProto listCacheDirectives(
+      RpcController controller, ListCacheDirectivesRequestProto request)
+          throws ServiceException {
+    try {
+      CacheDirectiveInfo filter =
+          PBHelper.convert(request.getFilter());
+      BatchedEntries<CacheDirectiveEntry> entries =
+        server.listCacheDirectives(request.getPrevId(), filter);
+      ListCacheDirectivesResponseProto.Builder builder =
+          ListCacheDirectivesResponseProto.newBuilder();
+      builder.setHasMore(entries.hasMore());
+      for (int i=0, n=entries.size(); i<n; i++) {
+        builder.addElements(PBHelper.convert(entries.get(i)));
+      }
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public AddCachePoolResponseProto addCachePool(RpcController controller,
+      AddCachePoolRequestProto request) throws ServiceException {
+    try {
+      server.addCachePool(PBHelper.convert(request.getInfo()));
+      return AddCachePoolResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
   
   
+  @Override
+  public ModifyCachePoolResponseProto modifyCachePool(RpcController controller,
+      ModifyCachePoolRequestProto request) throws ServiceException {
+    try {
+      server.modifyCachePool(PBHelper.convert(request.getInfo()));
+      return ModifyCachePoolResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RemoveCachePoolResponseProto removeCachePool(RpcController controller,
+      RemoveCachePoolRequestProto request) throws ServiceException {
+    try {
+      server.removeCachePool(request.getPoolName());
+      return RemoveCachePoolResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ListCachePoolsResponseProto listCachePools(RpcController controller,
+      ListCachePoolsRequestProto request) throws ServiceException {
+    try {
+      BatchedEntries<CachePoolEntry> entries =
+        server.listCachePools(request.getPrevPoolName());
+      ListCachePoolsResponseProto.Builder responseBuilder =
+        ListCachePoolsResponseProto.newBuilder();
+      responseBuilder.setHasMore(entries.hasMore());
+      for (int i=0, n=entries.size(); i<n; i++) {
+        responseBuilder.addEntries(PBHelper.convert(entries.get(i)));
+      }
+      return responseBuilder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }
 }

+ 179 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -21,9 +21,12 @@ 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 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.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;
@@ -33,6 +36,9 @@ import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@@ -46,13 +52,17 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
@@ -87,11 +97,19 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
@@ -985,4 +1003,165 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
+
+  @Override
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    try {
+      AddCacheDirectiveRequestProto.Builder builder =
+          AddCacheDirectiveRequestProto.newBuilder().
+              setInfo(PBHelper.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelper.convertCacheFlags(flags));
+      }
+      return rpcProxy.addCacheDirective(null, builder.build()).getId();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    try {
+      ModifyCacheDirectiveRequestProto.Builder builder =
+          ModifyCacheDirectiveRequestProto.newBuilder().
+              setInfo(PBHelper.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelper.convertCacheFlags(flags));
+      }
+      rpcProxy.modifyCacheDirective(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeCacheDirective(long id)
+      throws IOException {
+    try {
+      rpcProxy.removeCacheDirective(null,
+          RemoveCacheDirectiveRequestProto.newBuilder().
+              setId(id).build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  private static class BatchedCacheEntries
+      implements BatchedEntries<CacheDirectiveEntry> {
+    private ListCacheDirectivesResponseProto response;
+
+    BatchedCacheEntries(
+        ListCacheDirectivesResponseProto response) {
+      this.response = response;
+    }
+
+    @Override
+    public CacheDirectiveEntry get(int i) {
+      return PBHelper.convert(response.getElements(i));
+    }
+
+    @Override
+    public int size() {
+      return response.getElementsCount();
+    }
+    
+    @Override
+    public boolean hasMore() {
+      return response.getHasMore();
+    }
+  }
+
+  @Override
+  public BatchedEntries<CacheDirectiveEntry>
+      listCacheDirectives(long prevId,
+          CacheDirectiveInfo filter) throws IOException {
+    if (filter == null) {
+      filter = new CacheDirectiveInfo.Builder().build();
+    }
+    try {
+      return new BatchedCacheEntries(
+        rpcProxy.listCacheDirectives(null,
+          ListCacheDirectivesRequestProto.newBuilder().
+            setPrevId(prevId).
+            setFilter(PBHelper.convert(filter)).
+            build()));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    AddCachePoolRequestProto.Builder builder = 
+        AddCachePoolRequestProto.newBuilder();
+    builder.setInfo(PBHelper.convert(info));
+    try {
+      rpcProxy.addCachePool(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void modifyCachePool(CachePoolInfo req) throws IOException {
+    ModifyCachePoolRequestProto.Builder builder = 
+        ModifyCachePoolRequestProto.newBuilder();
+    builder.setInfo(PBHelper.convert(req));
+    try {
+      rpcProxy.modifyCachePool(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeCachePool(String cachePoolName) throws IOException {
+    try {
+      rpcProxy.removeCachePool(null, 
+          RemoveCachePoolRequestProto.newBuilder().
+            setPoolName(cachePoolName).build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  private static class BatchedCachePoolEntries
+    implements BatchedEntries<CachePoolEntry> {
+      private final ListCachePoolsResponseProto proto;
+    
+    public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
+      this.proto = proto;
+    }
+      
+    @Override
+    public CachePoolEntry get(int i) {
+      CachePoolEntryProto elem = proto.getEntries(i);
+      return PBHelper.convert(elem);
+    }
+
+    @Override
+    public int size() {
+      return proto.getEntriesCount();
+    }
+    
+    @Override
+    public boolean hasMore() {
+      return proto.getHasMore();
+    }
+  }
+
+  @Override
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+      throws IOException {
+    try {
+      return new BatchedCachePoolEntries(
+        rpcProxy.listCachePools(null,
+          ListCachePoolsRequestProto.newBuilder().
+            setPrevPoolName(prevKey).build()));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }
 }

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

@@ -22,6 +22,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.util.HashMap;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
@@ -36,6 +37,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
@@ -152,8 +155,9 @@ public class DatanodeProtocolClientSideTranslatorPB implements
 
 
   @Override
   @Override
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
-      StorageReport[] reports, int xmitsInProgress, int xceiverCount,
-      int failedVolumes) throws IOException {
+      StorageReport[] reports, long cacheCapacity, long cacheUsed,
+          int xmitsInProgress, int xceiverCount, int failedVolumes)
+              throws IOException {
     HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
     HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
         .setRegistration(PBHelper.convert(registration))
         .setRegistration(PBHelper.convert(registration))
         .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
         .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
@@ -161,7 +165,12 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     for (StorageReport r : reports) {
     for (StorageReport r : reports) {
       builder.addReports(PBHelper.convert(r));
       builder.addReports(PBHelper.convert(r));
     }
     }
-    
+    if (cacheCapacity != 0) {
+      builder.setCacheCapacity(cacheCapacity);
+    }
+    if (cacheUsed != 0) {
+      builder.setCacheUsed(cacheUsed);
+    }
     HeartbeatResponseProto resp;
     HeartbeatResponseProto resp;
     try {
     try {
       resp = rpcProxy.sendHeartbeat(NULL_CONTROLLER, builder.build());
       resp = rpcProxy.sendHeartbeat(NULL_CONTROLLER, builder.build());
@@ -202,6 +211,29 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
     return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
   }
   }
 
 
+  @Override
+  public DatanodeCommand cacheReport(DatanodeRegistration registration,
+      String poolId, List<Long> blockIds) throws IOException {
+    CacheReportRequestProto.Builder builder =
+        CacheReportRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration))
+        .setBlockPoolId(poolId);
+    for (Long blockId : blockIds) {
+      builder.addBlocks(blockId);
+    }
+    
+    CacheReportResponseProto resp;
+    try {
+      resp = rpcProxy.cacheReport(NULL_CONTROLLER, builder.build());
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+    if (resp.hasCmd()) {
+      return PBHelper.convert(resp.getCmd());
+    }
+    return null;
+  }
+
   @Override
   @Override
   public void blockReceivedAndDeleted(DatanodeRegistration registration,
   public void blockReceivedAndDeleted(DatanodeRegistration registration,
       String poolId, StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks)
       String poolId, StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks)

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

@@ -27,6 +27,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceive
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
@@ -102,8 +104,9 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       final StorageReport[] report = PBHelper.convertStorageReports(
       final StorageReport[] report = PBHelper.convertStorageReports(
           request.getReportsList());
           request.getReportsList());
       response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
       response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
-          report, request.getXmitsInProgress(), request.getXceiverCount(),
-          request.getFailedVolumes());
+          report, request.getCacheCapacity(), request.getCacheUsed(),
+          request.getXmitsInProgress(),
+          request.getXceiverCount(), request.getFailedVolumes());
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);
     }
     }
@@ -152,6 +155,27 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     return builder.build();
     return builder.build();
   }
   }
 
 
+  @Override
+  public CacheReportResponseProto cacheReport(RpcController controller,
+      CacheReportRequestProto request) throws ServiceException {
+    DatanodeCommand cmd = null;
+    try {
+      cmd = impl.cacheReport(
+          PBHelper.convert(request.getRegistration()),
+          request.getBlockPoolId(),
+          request.getBlocksList());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    CacheReportResponseProto.Builder builder =
+        CacheReportResponseProto.newBuilder();
+    if (cmd != null) {
+      builder.setCmd(PBHelper.convert(cmd));
+    }
+    return builder.build();
+  }
+
+
   @Override
   @Override
   public BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
   public BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
       RpcController controller, BlockReceivedAndDeletedRequestProto request)
       RpcController controller, BlockReceivedAndDeletedRequestProto request)

+ 304 - 30
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -17,6 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.protocolPB;
 package org.apache.hadoop.hdfs.protocolPB;
 
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.io.EOFException;
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
@@ -26,18 +28,26 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+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;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolStats;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@@ -52,12 +62,21 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
@@ -121,6 +140,7 @@ import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -151,7 +171,9 @@ import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
 
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
+import com.google.common.primitives.Shorts;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedInputStream;
 
 
@@ -482,27 +504,14 @@ public class PBHelper {
         PBHelper.convert(di.getId()),
         PBHelper.convert(di.getId()),
         di.hasLocation() ? di.getLocation() : null , 
         di.hasLocation() ? di.getLocation() : null , 
         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
-        di.getBlockPoolUsed()  ,  di.getLastUpdate() , di.getXceiverCount() ,
+        di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
+        di.getLastUpdate(), di.getXceiverCount(),
         PBHelper.convert(di.getAdminState())); 
         PBHelper.convert(di.getAdminState())); 
   }
   }
   
   
   static public DatanodeInfoProto convertDatanodeInfo(DatanodeInfo di) {
   static public DatanodeInfoProto convertDatanodeInfo(DatanodeInfo di) {
     if (di == null) return null;
     if (di == null) return null;
-    DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
-    if (di.getNetworkLocation() != null) {
-      builder.setLocation(di.getNetworkLocation());
-    }
-        
-    return builder.
-     setId(PBHelper.convert((DatanodeID) di)).
-     setCapacity(di.getCapacity()).
-     setDfsUsed(di.getDfsUsed()).
-     setRemaining(di.getRemaining()).
-     setBlockPoolUsed(di.getBlockPoolUsed()).
-     setLastUpdate(di.getLastUpdate()).
-     setXceiverCount(di.getXceiverCount()).
-     setAdminState(PBHelper.convert(di.getAdminState())).
-     build();     
+    return convert(di);
   }
   }
   
   
   
   
@@ -546,15 +555,20 @@ public class PBHelper {
   
   
   public static DatanodeInfoProto convert(DatanodeInfo info) {
   public static DatanodeInfoProto convert(DatanodeInfo info) {
     DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
     DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
-    builder.setBlockPoolUsed(info.getBlockPoolUsed());
-    builder.setAdminState(PBHelper.convert(info.getAdminState()));
-    builder.setCapacity(info.getCapacity())
-        .setDfsUsed(info.getDfsUsed())
+    if (info.getNetworkLocation() != null) {
+      builder.setLocation(info.getNetworkLocation());
+    }
+    builder
         .setId(PBHelper.convert((DatanodeID)info))
         .setId(PBHelper.convert((DatanodeID)info))
-        .setLastUpdate(info.getLastUpdate())
-        .setLocation(info.getNetworkLocation())
+        .setCapacity(info.getCapacity())
+        .setDfsUsed(info.getDfsUsed())
         .setRemaining(info.getRemaining())
         .setRemaining(info.getRemaining())
+        .setBlockPoolUsed(info.getBlockPoolUsed())
+        .setCacheCapacity(info.getCacheCapacity())
+        .setCacheUsed(info.getCacheUsed())
+        .setLastUpdate(info.getLastUpdate())
         .setXceiverCount(info.getXceiverCount())
         .setXceiverCount(info.getXceiverCount())
+        .setAdminState(PBHelper.convert(info.getAdminState()))
         .build();
         .build();
     return builder.build();
     return builder.build();
   }
   }
@@ -575,9 +589,20 @@ public class PBHelper {
     if (b == null) return null;
     if (b == null) return null;
     Builder builder = LocatedBlockProto.newBuilder();
     Builder builder = LocatedBlockProto.newBuilder();
     DatanodeInfo[] locs = b.getLocations();
     DatanodeInfo[] locs = b.getLocations();
+    List<DatanodeInfo> cachedLocs =
+        Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
     for (int i = 0; i < locs.length; i++) {
     for (int i = 0; i < locs.length; i++) {
-      builder.addLocs(i, PBHelper.convert(locs[i]));
+      DatanodeInfo loc = locs[i];
+      builder.addLocs(i, PBHelper.convert(loc));
+      boolean locIsCached = cachedLocs.contains(loc);
+      builder.addIsCached(locIsCached);
+      if (locIsCached) {
+        cachedLocs.remove(loc);
+      }
     }
     }
+    Preconditions.checkArgument(cachedLocs.size() == 0,
+        "Found additional cached replica locations that are not in the set of"
+        + " storage-backed locations!");
 
 
     StorageType[] storageTypes = b.getStorageTypes();
     StorageType[] storageTypes = b.getStorageTypes();
     if (storageTypes != null) {
     if (storageTypes != null) {
@@ -621,9 +646,20 @@ public class PBHelper {
       storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
       storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
     }
     }
 
 
+    // Set values from the isCached list, re-using references from loc
+    List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
+    List<Boolean> isCachedList = proto.getIsCachedList();
+    for (int i=0; i<isCachedList.size(); i++) {
+      if (isCachedList.get(i)) {
+        cachedLocs.add(targets[i]);
+      }
+    }
+
     LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
     LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
-        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt());
+        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
+        cachedLocs.toArray(new DatanodeInfo[0]));
     lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
     lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
+
     return lb;
     return lb;
   }
   }
 
 
@@ -712,6 +748,8 @@ public class PBHelper {
       return PBHelper.convert(proto.getKeyUpdateCmd());
       return PBHelper.convert(proto.getKeyUpdateCmd());
     case RegisterCommand:
     case RegisterCommand:
       return REG_CMD;
       return REG_CMD;
+    case BlockIdCommand:
+      return PBHelper.convert(proto.getBlkIdCmd());
     }
     }
     return null;
     return null;
   }
   }
@@ -765,6 +803,26 @@ public class PBHelper {
            .addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs()));
            .addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs()));
     return builder.build();
     return builder.build();
   }
   }
+  
+  public static BlockIdCommandProto convert(BlockIdCommand cmd) {
+    BlockIdCommandProto.Builder builder = BlockIdCommandProto.newBuilder()
+        .setBlockPoolId(cmd.getBlockPoolId());
+    switch (cmd.getAction()) {
+    case DatanodeProtocol.DNA_CACHE:
+      builder.setAction(BlockIdCommandProto.Action.CACHE);
+      break;
+    case DatanodeProtocol.DNA_UNCACHE:
+      builder.setAction(BlockIdCommandProto.Action.UNCACHE);
+      break;
+    default:
+      throw new AssertionError("Invalid action");
+    }
+    long[] blockIds = cmd.getBlockIds();
+    for (int i = 0; i < blockIds.length; i++) {
+      builder.addBlockIds(blockIds[i]);
+    }
+    return builder.build();
+  }
 
 
   private static List<DatanodeInfosProto> convert(DatanodeInfo[][] targets) {
   private static List<DatanodeInfosProto> convert(DatanodeInfo[][] targets) {
     DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
     DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
@@ -817,8 +875,13 @@ public class PBHelper {
     case DatanodeProtocol.DNA_TRANSFER:
     case DatanodeProtocol.DNA_TRANSFER:
     case DatanodeProtocol.DNA_INVALIDATE:
     case DatanodeProtocol.DNA_INVALIDATE:
     case DatanodeProtocol.DNA_SHUTDOWN:
     case DatanodeProtocol.DNA_SHUTDOWN:
-      builder.setCmdType(DatanodeCommandProto.Type.BlockCommand).setBlkCmd(
-          PBHelper.convert((BlockCommand) datanodeCommand));
+      builder.setCmdType(DatanodeCommandProto.Type.BlockCommand).
+        setBlkCmd(PBHelper.convert((BlockCommand) datanodeCommand));
+      break;
+    case DatanodeProtocol.DNA_CACHE:
+    case DatanodeProtocol.DNA_UNCACHE:
+      builder.setCmdType(DatanodeCommandProto.Type.BlockIdCommand).
+        setBlkIdCmd(PBHelper.convert((BlockIdCommand) datanodeCommand));
       break;
       break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
     default:
@@ -877,11 +940,33 @@ public class PBHelper {
     case SHUTDOWN:
     case SHUTDOWN:
       action = DatanodeProtocol.DNA_SHUTDOWN;
       action = DatanodeProtocol.DNA_SHUTDOWN;
       break;
       break;
+    default:
+      throw new AssertionError("Unknown action type: " + blkCmd.getAction());
     }
     }
     return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets,
     return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets,
         targetStorageIDs);
         targetStorageIDs);
   }
   }
 
 
+  public static BlockIdCommand convert(BlockIdCommandProto blkIdCmd) {
+    int numBlockIds = blkIdCmd.getBlockIdsCount();
+    long blockIds[] = new long[numBlockIds];
+    for (int i = 0; i < numBlockIds; i++) {
+      blockIds[i] = blkIdCmd.getBlockIds(i);
+    }
+    int action = DatanodeProtocol.DNA_UNKNOWN;
+    switch (blkIdCmd.getAction()) {
+    case CACHE:
+      action = DatanodeProtocol.DNA_CACHE;
+      break;
+    case UNCACHE:
+      action = DatanodeProtocol.DNA_UNCACHE;
+      break;
+    default:
+      throw new AssertionError("Unknown action type: " + blkIdCmd.getAction());
+    }
+    return new BlockIdCommand(action, blkIdCmd.getBlockPoolId(), blockIds);
+  }
+
   public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
   public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
     List<DatanodeInfoProto> proto = datanodeInfosProto.getDatanodesList();
     List<DatanodeInfoProto> proto = datanodeInfosProto.getDatanodesList();
     DatanodeInfo[] infos = new DatanodeInfo[proto.size()];
     DatanodeInfo[] infos = new DatanodeInfo[proto.size()];
@@ -1090,7 +1175,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) {
@@ -1105,7 +1190,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;
@@ -1455,11 +1556,12 @@ public class PBHelper {
   }
   }
 
 
   public static StorageReportProto convert(StorageReport r) {
   public static StorageReportProto convert(StorageReport r) {
-    return StorageReportProto.newBuilder()
+    StorageReportProto.Builder builder = StorageReportProto.newBuilder()
         .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
         .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
         .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
         .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
         .setStorageUuid(r.getStorage().getStorageID())
         .setStorageUuid(r.getStorage().getStorageID())
-        .setStorage(convert(r.getStorage())).build();
+        .setStorage(convert(r.getStorage()));
+    return builder.build();
   }
   }
 
 
   public static StorageReport convert(StorageReportProto p) {
   public static StorageReport convert(StorageReportProto p) {
@@ -1598,6 +1700,178 @@ public class PBHelper {
     return DataChecksum.Type.valueOf(type.getNumber());
     return DataChecksum.Type.valueOf(type.getNumber());
   }
   }
 
 
+  public static CacheDirectiveInfoProto convert
+      (CacheDirectiveInfo info) {
+    CacheDirectiveInfoProto.Builder builder = 
+        CacheDirectiveInfoProto.newBuilder();
+    if (info.getId() != null) {
+      builder.setId(info.getId());
+    }
+    if (info.getPath() != null) {
+      builder.setPath(info.getPath().toUri().getPath());
+    }
+    if (info.getReplication() != null) {
+      builder.setReplication(info.getReplication());
+    }
+    if (info.getPool() != null) {
+      builder.setPool(info.getPool());
+    }
+    if (info.getExpiration() != null) {
+      builder.setExpiration(convert(info.getExpiration()));
+    }
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfo convert
+      (CacheDirectiveInfoProto proto) {
+    CacheDirectiveInfo.Builder builder =
+        new CacheDirectiveInfo.Builder();
+    if (proto.hasId()) {
+      builder.setId(proto.getId());
+    }
+    if (proto.hasPath()) {
+      builder.setPath(new Path(proto.getPath()));
+    }
+    if (proto.hasReplication()) {
+      builder.setReplication(Shorts.checkedCast(
+          proto.getReplication()));
+    }
+    if (proto.hasPool()) {
+      builder.setPool(proto.getPool());
+    }
+    if (proto.hasExpiration()) {
+      builder.setExpiration(convert(proto.getExpiration()));
+    }
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfoExpirationProto convert(
+      CacheDirectiveInfo.Expiration expiration) {
+    return CacheDirectiveInfoExpirationProto.newBuilder()
+        .setIsRelative(expiration.isRelative())
+        .setMillis(expiration.getMillis())
+        .build();
+  }
+
+  public static CacheDirectiveInfo.Expiration convert(
+      CacheDirectiveInfoExpirationProto proto) {
+    if (proto.getIsRelative()) {
+      return CacheDirectiveInfo.Expiration.newRelative(proto.getMillis());
+    }
+    return CacheDirectiveInfo.Expiration.newAbsolute(proto.getMillis());
+  }
+
+  public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
+    CacheDirectiveStatsProto.Builder builder = 
+        CacheDirectiveStatsProto.newBuilder();
+    builder.setBytesNeeded(stats.getBytesNeeded());
+    builder.setBytesCached(stats.getBytesCached());
+    builder.setFilesNeeded(stats.getFilesNeeded());
+    builder.setFilesCached(stats.getFilesCached());
+    builder.setHasExpired(stats.hasExpired());
+    return builder.build();
+  }
+  
+  public static CacheDirectiveStats convert(CacheDirectiveStatsProto proto) {
+    CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
+    builder.setBytesNeeded(proto.getBytesNeeded());
+    builder.setBytesCached(proto.getBytesCached());
+    builder.setFilesNeeded(proto.getFilesNeeded());
+    builder.setFilesCached(proto.getFilesCached());
+    builder.setHasExpired(proto.getHasExpired());
+    return builder.build();
+  }
+
+  public static CacheDirectiveEntryProto convert(CacheDirectiveEntry entry) {
+    CacheDirectiveEntryProto.Builder builder = 
+        CacheDirectiveEntryProto.newBuilder();
+    builder.setInfo(PBHelper.convert(entry.getInfo()));
+    builder.setStats(PBHelper.convert(entry.getStats()));
+    return builder.build();
+  }
+  
+  public static CacheDirectiveEntry convert(CacheDirectiveEntryProto proto) {
+    CacheDirectiveInfo info = PBHelper.convert(proto.getInfo());
+    CacheDirectiveStats stats = PBHelper.convert(proto.getStats());
+    return new CacheDirectiveEntry(info, stats);
+  }
+
+  public static CachePoolInfoProto convert(CachePoolInfo info) {
+    CachePoolInfoProto.Builder builder = CachePoolInfoProto.newBuilder();
+    builder.setPoolName(info.getPoolName());
+    if (info.getOwnerName() != null) {
+      builder.setOwnerName(info.getOwnerName());
+    }
+    if (info.getGroupName() != null) {
+      builder.setGroupName(info.getGroupName());
+    }
+    if (info.getMode() != null) {
+      builder.setMode(info.getMode().toShort());
+    }
+    if (info.getLimit() != null) {
+      builder.setLimit(info.getLimit());
+    }
+    if (info.getMaxRelativeExpiryMs() != null) {
+      builder.setMaxRelativeExpiry(info.getMaxRelativeExpiryMs());
+    }
+    return builder.build();
+  }
+
+  public static CachePoolInfo convert (CachePoolInfoProto proto) {
+    // Pool name is a required field, the rest are optional
+    String poolName = checkNotNull(proto.getPoolName());
+    CachePoolInfo info = new CachePoolInfo(poolName);
+    if (proto.hasOwnerName()) {
+        info.setOwnerName(proto.getOwnerName());
+    }
+    if (proto.hasGroupName()) {
+      info.setGroupName(proto.getGroupName());
+    }
+    if (proto.hasMode()) {
+      info.setMode(new FsPermission((short)proto.getMode()));
+    }
+    if (proto.hasLimit())  {
+      info.setLimit(proto.getLimit());
+    }
+    if (proto.hasMaxRelativeExpiry()) {
+      info.setMaxRelativeExpiryMs(proto.getMaxRelativeExpiry());
+    }
+    return info;
+  }
+
+  public static CachePoolStatsProto convert(CachePoolStats stats) {
+    CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
+    builder.setBytesNeeded(stats.getBytesNeeded());
+    builder.setBytesCached(stats.getBytesCached());
+    builder.setBytesOverlimit(stats.getBytesOverlimit());
+    builder.setFilesNeeded(stats.getFilesNeeded());
+    builder.setFilesCached(stats.getFilesCached());
+    return builder.build();
+  }
+
+  public static CachePoolStats convert (CachePoolStatsProto proto) {
+    CachePoolStats.Builder builder = new CachePoolStats.Builder();
+    builder.setBytesNeeded(proto.getBytesNeeded());
+    builder.setBytesCached(proto.getBytesCached());
+    builder.setBytesOverlimit(proto.getBytesOverlimit());
+    builder.setFilesNeeded(proto.getFilesNeeded());
+    builder.setFilesCached(proto.getFilesCached());
+    return builder.build();
+  }
+
+  public static CachePoolEntryProto convert(CachePoolEntry entry) {
+    CachePoolEntryProto.Builder builder = CachePoolEntryProto.newBuilder();
+    builder.setInfo(PBHelper.convert(entry.getInfo()));
+    builder.setStats(PBHelper.convert(entry.getStats()));
+    return builder.build();
+  }
+
+  public static CachePoolEntry convert (CachePoolEntryProto proto) {
+    CachePoolInfo info = PBHelper.convert(proto.getInfo());
+    CachePoolStats stats = PBHelper.convert(proto.getStats());
+    return new CachePoolEntry(info, stats);
+  }
+  
   public static HdfsProtos.ChecksumTypeProto convert(DataChecksum.Type type) {
   public static HdfsProtos.ChecksumTypeProto convert(DataChecksum.Type type) {
     return HdfsProtos.ChecksumTypeProto.valueOf(type.id);
     return HdfsProtos.ChecksumTypeProto.valueOf(type.id);
   }
   }

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

@@ -158,7 +158,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     return info;
     return info;
   }
   }
 
 
-  int getCapacity() {
+  public int getCapacity() {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert this.triplets != null : "BlockInfo is not initialized";
     assert triplets.length % 3 == 0 : "Malformed BlockInfo";
     assert triplets.length % 3 == 0 : "Malformed BlockInfo";
     return triplets.length / 3;
     return triplets.length / 3;

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

@@ -3180,6 +3180,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
         UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }
   }
 
 
+  /**
+   * Get the replicas which are corrupt for a given block.
+   */
+  public Collection<DatanodeDescriptor> getCorruptReplicas(Block block) {
+    return corruptReplicas.getNodes(block);
+  }
+
   /** @return the size of UnderReplicatedBlocks */
   /** @return the size of UnderReplicatedBlocks */
   public int numOfUnderReplicatedBlocks() {
   public int numOfUnderReplicatedBlocks() {
     return neededReplications.size();
     return neededReplications.size();

+ 774 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java

@@ -0,0 +1,774 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.CacheDirective;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.CacheManager;
+import org.apache.hadoop.hdfs.server.namenode.CachePool;
+import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.apache.hadoop.util.GSet;
+import org.apache.hadoop.util.Time;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Scans the namesystem, scheduling blocks to be cached as appropriate.
+ *
+ * The CacheReplicationMonitor does a full scan when the NameNode first
+ * starts up, and at configurable intervals afterwards.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+public class CacheReplicationMonitor extends Thread implements Closeable {
+
+  private static final Log LOG =
+      LogFactory.getLog(CacheReplicationMonitor.class);
+
+  private final FSNamesystem namesystem;
+
+  private final BlockManager blockManager;
+
+  private final CacheManager cacheManager;
+
+  private final GSet<CachedBlock, CachedBlock> cachedBlocks;
+
+  /**
+   * Pseudorandom number source
+   */
+  private static final Random random = new Random();
+
+  /**
+   * The interval at which we scan the namesystem for caching changes.
+   */
+  private final long intervalMs;
+
+  /**
+   * The CacheReplicationMonitor (CRM) lock. Used to synchronize starting and
+   * waiting for rescan operations.
+   */
+  private final ReentrantLock lock;
+
+  /**
+   * Notifies the scan thread that an immediate rescan is needed.
+   */
+  private final Condition doRescan;
+
+  /**
+   * Notifies waiting threads that a rescan has finished.
+   */
+  private final Condition scanFinished;
+
+  /**
+   * Whether there are pending CacheManager operations that necessitate a
+   * CacheReplicationMonitor rescan. Protected by the CRM lock.
+   */
+  private boolean needsRescan = true;
+
+  /**
+   * Whether we are currently doing a rescan. Protected by the CRM lock.
+   */
+  private boolean isScanning = false;
+
+  /**
+   * The number of rescans completed. Used to wait for scans to finish.
+   * Protected by the CacheReplicationMonitor lock.
+   */
+  private long scanCount = 0;
+
+  /**
+   * True if this monitor should terminate. Protected by the CRM lock.
+   */
+  private boolean shutdown = false;
+
+  /**
+   * Mark status of the current scan.
+   */
+  private boolean mark = false;
+
+  /**
+   * Cache directives found in the previous scan.
+   */
+  private int scannedDirectives;
+
+  /**
+   * Blocks found in the previous scan.
+   */
+  private long scannedBlocks;
+
+  public CacheReplicationMonitor(FSNamesystem namesystem,
+      CacheManager cacheManager, long intervalMs, ReentrantLock lock) {
+    this.namesystem = namesystem;
+    this.blockManager = namesystem.getBlockManager();
+    this.cacheManager = cacheManager;
+    this.cachedBlocks = cacheManager.getCachedBlocks();
+    this.intervalMs = intervalMs;
+    this.lock = lock;
+    this.doRescan = this.lock.newCondition();
+    this.scanFinished = this.lock.newCondition();
+  }
+
+  @Override
+  public void run() {
+    long startTimeMs = 0;
+    Thread.currentThread().setName("CacheReplicationMonitor(" +
+        System.identityHashCode(this) + ")");
+    LOG.info("Starting CacheReplicationMonitor with interval " +
+             intervalMs + " milliseconds");
+    try {
+      long curTimeMs = Time.monotonicNow();
+      while (true) {
+        lock.lock();
+        try {
+          while (true) {
+            if (shutdown) {
+              LOG.info("Shutting down CacheReplicationMonitor");
+              return;
+            }
+            if (needsRescan) {
+              LOG.info("Rescanning because of pending operations");
+              break;
+            }
+            long delta = (startTimeMs + intervalMs) - curTimeMs;
+            if (delta <= 0) {
+              LOG.info("Rescanning after " + (curTimeMs - startTimeMs) +
+                  " milliseconds");
+              break;
+            }
+            doRescan.await(delta, TimeUnit.MILLISECONDS);
+            curTimeMs = Time.monotonicNow();
+          }
+          isScanning = true;
+          needsRescan = false;
+        } finally {
+          lock.unlock();
+        }
+        startTimeMs = curTimeMs;
+        mark = !mark;
+        rescan();
+        curTimeMs = Time.monotonicNow();
+        // Update synchronization-related variables.
+        lock.lock();
+        try {
+          isScanning = false;
+          scanCount++;
+          scanFinished.signalAll();
+        } finally {
+          lock.unlock();
+        }
+        LOG.info("Scanned " + scannedDirectives + " directive(s) and " +
+            scannedBlocks + " block(s) in " + (curTimeMs - startTimeMs) + " " +
+            "millisecond(s).");
+      }
+    } catch (InterruptedException e) {
+      LOG.info("Shutting down CacheReplicationMonitor.");
+      return;
+    } catch (Throwable t) {
+      LOG.fatal("Thread exiting", t);
+      terminate(1, t);
+    }
+  }
+
+  /**
+   * 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 waitForRescanIfNeeded() {
+    Preconditions.checkArgument(!namesystem.hasWriteLock(),
+        "Must not hold the FSN write lock when waiting for a rescan.");
+    Preconditions.checkArgument(lock.isHeldByCurrentThread(),
+        "Must hold the CRM lock when waiting for a rescan.");
+    if (!needsRescan) {
+      return;
+    }
+    // If no scan is already ongoing, mark the CRM as dirty and kick
+    if (!isScanning) {
+      doRescan.signal();
+    }
+    // Wait until the scan finishes and the count advances
+    final long startCount = scanCount;
+    while ((!shutdown) && (startCount >= scanCount)) {
+      try {
+        scanFinished.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for CacheReplicationMonitor"
+            + " rescan", e);
+        break;
+      }
+    }
+  }
+
+  /**
+   * Indicates to the CacheReplicationMonitor that there have been CacheManager
+   * changes that require a rescan.
+   */
+  public void setNeedsRescan() {
+    Preconditions.checkArgument(lock.isHeldByCurrentThread(),
+        "Must hold the CRM lock when setting the needsRescan bit.");
+    this.needsRescan = true;
+  }
+
+  /**
+   * Shut down the monitor thread.
+   */
+  @Override
+  public void close() throws IOException {
+    Preconditions.checkArgument(namesystem.hasWriteLock());
+    lock.lock();
+    try {
+      if (shutdown) return;
+      // Since we hold both the FSN write lock and the CRM lock here,
+      // we know that the CRM thread cannot be currently modifying
+      // the cache manager state while we're closing it.
+      // Since the CRM thread checks the value of 'shutdown' after waiting
+      // for a lock, we know that the thread will not modify the cache
+      // manager state after this point.
+      shutdown = true;
+      doRescan.signalAll();
+      scanFinished.signalAll();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  private void rescan() throws InterruptedException {
+    scannedDirectives = 0;
+    scannedBlocks = 0;
+    namesystem.writeLock();
+    try {
+      if (shutdown) {
+        throw new InterruptedException("CacheReplicationMonitor was " +
+            "shut down.");
+      }
+      resetStatistics();
+      rescanCacheDirectives();
+      rescanCachedBlockMap();
+      blockManager.getDatanodeManager().resetLastCachingDirectiveSentTime();
+    } finally {
+      namesystem.writeUnlock();
+    }
+  }
+
+  private void resetStatistics() {
+    for (CachePool pool: cacheManager.getCachePools()) {
+      pool.resetStatistics();
+    }
+    for (CacheDirective directive: cacheManager.getCacheDirectives()) {
+      directive.resetStatistics();
+    }
+  }
+
+  /**
+   * Scan all CacheDirectives.  Use the information to figure out
+   * what cache replication factor each block should have.
+   */
+  private void rescanCacheDirectives() {
+    FSDirectory fsDir = namesystem.getFSDirectory();
+    final long now = new Date().getTime();
+    for (CacheDirective directive : cacheManager.getCacheDirectives()) {
+      // Skip processing this entry if it has expired
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Directive expiry is at " + directive.getExpiryTime());
+      }
+      if (directive.getExpiryTime() > 0 && directive.getExpiryTime() <= now) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Skipping directive id " + directive.getId()
+              + " because it has expired (" + directive.getExpiryTime() + "<="
+              + now + ")");
+        }
+        continue;
+      }
+      scannedDirectives++;
+      String path = directive.getPath();
+      INode node;
+      try {
+        node = fsDir.getINode(path);
+      } catch (UnresolvedLinkException e) {
+        // We don't cache through symlinks
+        continue;
+      }
+      if (node == null)  {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("No inode found at " + path);
+        }
+      } else if (node.isDirectory()) {
+        INodeDirectory dir = node.asDirectory();
+        ReadOnlyList<INode> children = dir.getChildrenList(null);
+        for (INode child : children) {
+          if (child.isFile()) {
+            rescanFile(directive, child.asFile());
+          }
+        }
+      } else if (node.isFile()) {
+        rescanFile(directive, node.asFile());
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Ignoring non-directory, non-file inode " + node +
+                    " found at " + path);
+        }
+      }
+    }
+  }
+  
+  /**
+   * Apply a CacheDirective to a file.
+   * 
+   * @param directive The CacheDirective to apply.
+   * @param file The file.
+   */
+  private void rescanFile(CacheDirective directive, INodeFile file) {
+    BlockInfo[] blockInfos = file.getBlocks();
+
+    // Increment the "needed" statistics
+    directive.addFilesNeeded(1);
+    // We don't cache UC blocks, don't add them to the total here
+    long neededTotal = file.computeFileSizeNotIncludingLastUcBlock() *
+        directive.getReplication();
+    directive.addBytesNeeded(neededTotal);
+
+    // The pool's bytesNeeded is incremented as we scan. If the demand
+    // thus far plus the demand of this file would exceed the pool's limit,
+    // do not cache this file.
+    CachePool pool = directive.getPool();
+    if (pool.getBytesNeeded() > pool.getLimit()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("Skipping directive id %d file %s because "
+            + "limit of pool %s would be exceeded (%d > %d)",
+            directive.getId(),
+            file.getFullPathName(),
+            pool.getPoolName(),
+            pool.getBytesNeeded(),
+            pool.getLimit()));
+      }
+      return;
+    }
+
+    long cachedTotal = 0;
+    for (BlockInfo blockInfo : blockInfos) {
+      if (!blockInfo.getBlockUCState().equals(BlockUCState.COMPLETE)) {
+        // We don't try to cache blocks that are under construction.
+        continue;
+      }
+      Block block = new Block(blockInfo.getBlockId());
+      CachedBlock ncblock = new CachedBlock(block.getBlockId(),
+          directive.getReplication(), mark);
+      CachedBlock ocblock = cachedBlocks.get(ncblock);
+      if (ocblock == null) {
+        cachedBlocks.put(ncblock);
+      } else {
+        // Update bytesUsed using the current replication levels.
+        // Assumptions: we assume that all the blocks are the same length
+        // on each datanode.  We can assume this because we're only caching
+        // blocks in state COMMITTED.
+        // Note that if two directives are caching the same block(s), they will
+        // both get them added to their bytesCached.
+        List<DatanodeDescriptor> cachedOn =
+            ocblock.getDatanodes(Type.CACHED);
+        long cachedByBlock = Math.min(cachedOn.size(),
+            directive.getReplication()) * blockInfo.getNumBytes();
+        cachedTotal += cachedByBlock;
+
+        if ((mark != ocblock.getMark()) ||
+            (ocblock.getReplication() < directive.getReplication())) {
+          //
+          // Overwrite the block's replication and mark in two cases:
+          //
+          // 1. If the mark on the CachedBlock is different from the mark for
+          // this scan, that means the block hasn't been updated during this
+          // scan, and we should overwrite whatever is there, since it is no
+          // longer valid.
+          //
+          // 2. If the replication in the CachedBlock is less than what the
+          // directive asks for, we want to increase the block's replication
+          // field to what the directive asks for.
+          //
+          ocblock.setReplicationAndMark(directive.getReplication(), mark);
+        }
+      }
+    }
+    // Increment the "cached" statistics
+    directive.addBytesCached(cachedTotal);
+    if (cachedTotal == neededTotal) {
+      directive.addFilesCached(1);
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Directive " + directive.getId() + " is caching " +
+          file.getFullPathName() + ": " + cachedTotal + "/" + neededTotal +
+          " bytes");
+    }
+  }
+
+  private String findReasonForNotCaching(CachedBlock cblock, 
+          BlockInfo blockInfo) {
+    if (blockInfo == null) {
+      // Somehow, a cache report with the block arrived, but the block
+      // reports from the DataNode haven't (yet?) described such a block.
+      // Alternately, the NameNode might have invalidated the block, but the
+      // DataNode hasn't caught up.  In any case, we want to tell the DN
+      // to uncache this.
+      return "not tracked by the BlockManager";
+    } else if (!blockInfo.isComplete()) {
+      // When a cached block changes state from complete to some other state
+      // on the DataNode (perhaps because of append), it will begin the
+      // uncaching process.  However, the uncaching process is not
+      // instantaneous, especially if clients have pinned the block.  So
+      // there may be a period of time when incomplete blocks remain cached
+      // on the DataNodes.
+      return "not complete";
+    } else if (cblock.getReplication() == 0) {
+      // Since 0 is not a valid value for a cache directive's replication
+      // field, seeing a replication of 0 on a CacheBlock means that it
+      // has never been reached by any sweep.
+      return "not needed by any directives";
+    } else if (cblock.getMark() != mark) { 
+      // Although the block was needed in the past, we didn't reach it during
+      // the current sweep.  Therefore, it doesn't need to be cached any more.
+      // Need to set the replication to 0 so it doesn't flip back to cached
+      // when the mark flips on the next scan
+      cblock.setReplicationAndMark((short)0, mark);
+      return "no longer needed by any directives";
+    }
+    return null;
+  }
+
+  /**
+   * Scan through the cached block map.
+   * Any blocks which are under-replicated should be assigned new Datanodes.
+   * Blocks that are over-replicated should be removed from Datanodes.
+   */
+  private void rescanCachedBlockMap() {
+    for (Iterator<CachedBlock> cbIter = cachedBlocks.iterator();
+        cbIter.hasNext(); ) {
+      scannedBlocks++;
+      CachedBlock cblock = cbIter.next();
+      List<DatanodeDescriptor> pendingCached =
+          cblock.getDatanodes(Type.PENDING_CACHED);
+      List<DatanodeDescriptor> cached =
+          cblock.getDatanodes(Type.CACHED);
+      List<DatanodeDescriptor> pendingUncached =
+          cblock.getDatanodes(Type.PENDING_UNCACHED);
+      // Remove nodes from PENDING_UNCACHED if they were actually uncached.
+      for (Iterator<DatanodeDescriptor> iter = pendingUncached.iterator();
+          iter.hasNext(); ) {
+        DatanodeDescriptor datanode = iter.next();
+        if (!cblock.isInList(datanode.getCached())) {
+          datanode.getPendingUncached().remove(cblock);
+          iter.remove();
+        }
+      }
+      BlockInfo blockInfo = blockManager.
+            getStoredBlock(new Block(cblock.getBlockId()));
+      String reason = findReasonForNotCaching(cblock, blockInfo);
+      int neededCached = 0;
+      if (reason != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("not caching " + cblock + " because it is " + reason);
+        }
+      } else {
+        neededCached = cblock.getReplication();
+      }
+      int numCached = cached.size();
+      if (numCached >= neededCached) {
+        // If we have enough replicas, drop all pending cached.
+        for (Iterator<DatanodeDescriptor> iter = pendingCached.iterator();
+            iter.hasNext(); ) {
+          DatanodeDescriptor datanode = iter.next();
+          datanode.getPendingCached().remove(cblock);
+          iter.remove();
+        }
+      }
+      if (numCached < neededCached) {
+        // If we don't have enough replicas, drop all pending uncached.
+        for (Iterator<DatanodeDescriptor> iter = pendingUncached.iterator();
+            iter.hasNext(); ) {
+          DatanodeDescriptor datanode = iter.next();
+          datanode.getPendingUncached().remove(cblock);
+          iter.remove();
+        }
+      }
+      int neededUncached = numCached -
+          (pendingUncached.size() + neededCached);
+      if (neededUncached > 0) {
+        addNewPendingUncached(neededUncached, cblock, cached,
+            pendingUncached);
+      } else {
+        int additionalCachedNeeded = neededCached -
+            (numCached + pendingCached.size());
+        if (additionalCachedNeeded > 0) {
+          addNewPendingCached(additionalCachedNeeded, cblock, cached,
+              pendingCached);
+        }
+      }
+      if ((neededCached == 0) &&
+          pendingUncached.isEmpty() &&
+          pendingCached.isEmpty()) {
+        // we have nothing more to do with this block.
+        cbIter.remove();
+      }
+    }
+  }
+
+  /**
+   * Add new entries to the PendingUncached list.
+   *
+   * @param neededUncached   The number of replicas that need to be uncached.
+   * @param cachedBlock      The block which needs to be uncached.
+   * @param cached           A list of DataNodes currently caching the block.
+   * @param pendingUncached  A list of DataNodes that will soon uncache the
+   *                         block.
+   */
+  private void addNewPendingUncached(int neededUncached,
+      CachedBlock cachedBlock, List<DatanodeDescriptor> cached,
+      List<DatanodeDescriptor> pendingUncached) {
+    // Figure out which replicas can be uncached.
+    LinkedList<DatanodeDescriptor> possibilities =
+        new LinkedList<DatanodeDescriptor>();
+    for (DatanodeDescriptor datanode : cached) {
+      if (!pendingUncached.contains(datanode)) {
+        possibilities.add(datanode);
+      }
+    }
+    while (neededUncached > 0) {
+      if (possibilities.isEmpty()) {
+        LOG.warn("Logic error: we're trying to uncache more replicas than " +
+            "actually exist for " + cachedBlock);
+        return;
+      }
+      DatanodeDescriptor datanode =
+        possibilities.remove(random.nextInt(possibilities.size()));
+      pendingUncached.add(datanode);
+      boolean added = datanode.getPendingUncached().add(cachedBlock);
+      assert added;
+      neededUncached--;
+    }
+  }
+  
+  /**
+   * Add new entries to the PendingCached list.
+   *
+   * @param neededCached     The number of replicas that need to be cached.
+   * @param cachedBlock      The block which needs to be cached.
+   * @param cached           A list of DataNodes currently caching the block.
+   * @param pendingCached    A list of DataNodes that will soon cache the
+   *                         block.
+   */
+  private void addNewPendingCached(final int neededCached,
+      CachedBlock cachedBlock, List<DatanodeDescriptor> cached,
+      List<DatanodeDescriptor> pendingCached) {
+    // To figure out which replicas can be cached, we consult the
+    // blocksMap.  We don't want to try to cache a corrupt replica, though.
+    BlockInfo blockInfo = blockManager.
+          getStoredBlock(new Block(cachedBlock.getBlockId()));
+    if (blockInfo == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not caching block " + cachedBlock + " because there " +
+            "is no record of it on the NameNode.");
+      }
+      return;
+    }
+    if (!blockInfo.isComplete()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not caching block " + cachedBlock + " because it " +
+            "is not yet complete.");
+      }
+      return;
+    }
+    // Filter the list of replicas to only the valid targets
+    List<DatanodeDescriptor> possibilities =
+        new LinkedList<DatanodeDescriptor>();
+    int numReplicas = blockInfo.getCapacity();
+    Collection<DatanodeDescriptor> corrupt =
+        blockManager.getCorruptReplicas(blockInfo);
+    int outOfCapacity = 0;
+    for (int i = 0; i < numReplicas; i++) {
+      DatanodeDescriptor datanode = blockInfo.getDatanode(i);
+      if (datanode == null) {
+        continue;
+      }
+      if (datanode.isDecommissioned() || datanode.isDecommissionInProgress()) {
+        continue;
+      }
+      if (corrupt != null && corrupt.contains(datanode)) {
+        continue;
+      }
+      if (pendingCached.contains(datanode) || cached.contains(datanode)) {
+        continue;
+      }
+      long pendingCapacity = datanode.getCacheRemaining();
+      // Subtract pending cached blocks from effective capacity
+      Iterator<CachedBlock> it = datanode.getPendingCached().iterator();
+      while (it.hasNext()) {
+        CachedBlock cBlock = it.next();
+        BlockInfo info =
+            blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
+        if (info != null) {
+          pendingCapacity -= info.getNumBytes();
+        }
+      }
+      it = datanode.getPendingUncached().iterator();
+      // Add pending uncached blocks from effective capacity
+      while (it.hasNext()) {
+        CachedBlock cBlock = it.next();
+        BlockInfo info =
+            blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
+        if (info != null) {
+          pendingCapacity += info.getNumBytes();
+        }
+      }
+      if (pendingCapacity < blockInfo.getNumBytes()) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Datanode " + datanode + " is not a valid possibility for"
+              + " block " + blockInfo.getBlockId() + " of size "
+              + blockInfo.getNumBytes() + " bytes, only has "
+              + datanode.getCacheRemaining() + " bytes of cache remaining.");
+        }
+        outOfCapacity++;
+        continue;
+      }
+      possibilities.add(datanode);
+    }
+    List<DatanodeDescriptor> chosen = chooseDatanodesForCaching(possibilities,
+        neededCached, blockManager.getDatanodeManager().getStaleInterval());
+    for (DatanodeDescriptor datanode : chosen) {
+      pendingCached.add(datanode);
+      boolean added = datanode.getPendingCached().add(cachedBlock);
+      assert added;
+    }
+    // We were unable to satisfy the requested replication factor
+    if (neededCached > chosen.size()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Only have " +
+            (cachedBlock.getReplication() - neededCached + chosen.size()) +
+            " of " + cachedBlock.getReplication() + " cached replicas for " +
+            cachedBlock + " (" + outOfCapacity + " nodes have insufficient " +
+            "capacity).");
+      }
+    }
+  }
+
+  /**
+   * Chooses datanode locations for caching from a list of valid possibilities.
+   * Non-stale nodes are chosen before stale nodes.
+   * 
+   * @param possibilities List of candidate datanodes
+   * @param neededCached Number of replicas needed
+   * @param staleInterval Age of a stale datanode
+   * @return A list of chosen datanodes
+   */
+  private static List<DatanodeDescriptor> chooseDatanodesForCaching(
+      final List<DatanodeDescriptor> possibilities, final int neededCached,
+      final long staleInterval) {
+    // Make a copy that we can modify
+    List<DatanodeDescriptor> targets =
+        new ArrayList<DatanodeDescriptor>(possibilities);
+    // Selected targets
+    List<DatanodeDescriptor> chosen = new LinkedList<DatanodeDescriptor>();
+
+    // Filter out stale datanodes
+    List<DatanodeDescriptor> stale = new LinkedList<DatanodeDescriptor>();
+    Iterator<DatanodeDescriptor> it = targets.iterator();
+    while (it.hasNext()) {
+      DatanodeDescriptor d = it.next();
+      if (d.isStale(staleInterval)) {
+        it.remove();
+        stale.add(d);
+      }
+    }
+    // Select targets
+    while (chosen.size() < neededCached) {
+      // Try to use stale nodes if we're out of non-stale nodes, else we're done
+      if (targets.isEmpty()) {
+        if (!stale.isEmpty()) {
+          targets = stale;
+        } else {
+          break;
+        }
+      }
+      // Select a random target
+      DatanodeDescriptor target =
+          chooseRandomDatanodeByRemainingCapacity(targets);
+      chosen.add(target);
+      targets.remove(target);
+    }
+    return chosen;
+  }
+
+  /**
+   * Choose a single datanode from the provided list of possible
+   * targets, weighted by the percentage of free space remaining on the node.
+   * 
+   * @return The chosen datanode
+   */
+  private static DatanodeDescriptor chooseRandomDatanodeByRemainingCapacity(
+      final List<DatanodeDescriptor> targets) {
+    // Use a weighted probability to choose the target datanode
+    float total = 0;
+    for (DatanodeDescriptor d : targets) {
+      total += d.getCacheRemainingPercent();
+    }
+    // Give each datanode a portion of keyspace equal to its relative weight
+    // [0, w1) selects d1, [w1, w2) selects d2, etc.
+    TreeMap<Integer, DatanodeDescriptor> lottery =
+        new TreeMap<Integer, DatanodeDescriptor>();
+    int offset = 0;
+    for (DatanodeDescriptor d : targets) {
+      // Since we're using floats, be paranoid about negative values
+      int weight =
+          Math.max(1, (int)((d.getCacheRemainingPercent() / total) * 1000000));
+      offset += weight;
+      lottery.put(offset, d);
+    }
+    // Choose a number from [0, offset), which is the total amount of weight,
+    // to select the winner
+    DatanodeDescriptor winner =
+        lottery.higherEntry(random.nextInt(offset)).getValue();
+    return winner;
+  }
+}

+ 113 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -17,7 +17,15 @@
  */
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -26,11 +34,15 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
+import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
 /**
  * This class extends the DatanodeInfo class with ephemeral information (eg
  * This class extends the DatanodeInfo class with ephemeral information (eg
  * health, capacity, what blocks are associated with the Datanode) that is
  * health, capacity, what blocks are associated with the Datanode) that is
@@ -99,6 +111,71 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final Map<String, DatanodeStorageInfo> storageMap = 
   private final Map<String, DatanodeStorageInfo> storageMap = 
       new HashMap<String, DatanodeStorageInfo>();
       new HashMap<String, DatanodeStorageInfo>();
 
 
+  /**
+   * A list of CachedBlock objects on this datanode.
+   */
+  public static class CachedBlocksList extends IntrusiveCollection<CachedBlock> {
+    public enum Type {
+      PENDING_CACHED,
+      CACHED,
+      PENDING_UNCACHED
+    }
+
+    private final DatanodeDescriptor datanode;
+
+    private final Type type;
+
+    CachedBlocksList(DatanodeDescriptor datanode, Type type) {
+      this.datanode = datanode;
+      this.type = type;
+    }
+
+    public DatanodeDescriptor getDatanode() {
+      return datanode;
+    }
+
+    public Type getType() {
+      return type;
+    }
+  }
+
+  /**
+   * The blocks which we want to cache on this DataNode.
+   */
+  private final CachedBlocksList pendingCached = 
+      new CachedBlocksList(this, CachedBlocksList.Type.PENDING_CACHED);
+
+  /**
+   * The blocks which we know are cached on this datanode.
+   * This list is updated by periodic cache reports.
+   */
+  private final CachedBlocksList cached = 
+      new CachedBlocksList(this, CachedBlocksList.Type.CACHED);
+
+  /**
+   * The blocks which we want to uncache on this DataNode.
+   */
+  private final CachedBlocksList pendingUncached = 
+      new CachedBlocksList(this, CachedBlocksList.Type.PENDING_UNCACHED);
+
+  public CachedBlocksList getPendingCached() {
+    return pendingCached;
+  }
+
+  public CachedBlocksList getCached() {
+    return cached;
+  }
+
+  public CachedBlocksList getPendingUncached() {
+    return pendingUncached;
+  }
+
+  /**
+   * The time when the last batch of caching directives was sent, in
+   * monotonic milliseconds.
+   */
+  private long lastCachingDirectiveSentTimeMs;
+
   // isAlive == heartbeats.contains(this)
   // isAlive == heartbeats.contains(this)
   // This is an optimization, because contains takes O(n) time on Arraylist
   // This is an optimization, because contains takes O(n) time on Arraylist
   public boolean isAlive = false;
   public boolean isAlive = false;
@@ -144,7 +221,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
    */
   public DatanodeDescriptor(DatanodeID nodeID) {
   public DatanodeDescriptor(DatanodeID nodeID) {
     super(nodeID);
     super(nodeID);
-    updateHeartbeat(StorageReport.EMPTY_ARRAY, 0, 0);
+    updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
   }
   }
 
 
   /**
   /**
@@ -155,7 +232,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   public DatanodeDescriptor(DatanodeID nodeID, 
   public DatanodeDescriptor(DatanodeID nodeID, 
                             String networkLocation) {
                             String networkLocation) {
     super(nodeID, networkLocation);
     super(nodeID, networkLocation);
-    updateHeartbeat(StorageReport.EMPTY_ARRAY, 0, 0);
+    updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
   }
   }
 
 
   /**
   /**
@@ -236,6 +313,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     setXceiverCount(0);
     setXceiverCount(0);
     this.invalidateBlocks.clear();
     this.invalidateBlocks.clear();
     this.volumeFailures = 0;
     this.volumeFailures = 0;
+    // pendingCached, cached, and pendingUncached are protected by the
+    // FSN lock.
+    this.pendingCached.clear();
+    this.cached.clear();
+    this.pendingUncached.clear();
   }
   }
   
   
   public void clearBlockQueues() {
   public void clearBlockQueues() {
@@ -244,6 +326,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
       this.recoverBlocks.clear();
       this.recoverBlocks.clear();
       this.replicateBlocks.clear();
       this.replicateBlocks.clear();
     }
     }
+    // pendingCached, cached, and pendingUncached are protected by the
+    // FSN lock.
+    this.pendingCached.clear();
+    this.cached.clear();
+    this.pendingUncached.clear();
   }
   }
 
 
   public int numBlocks() {
   public int numBlocks() {
@@ -257,13 +344,15 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
   /**
    * Updates stats from datanode heartbeat.
    * Updates stats from datanode heartbeat.
    */
    */
-  public void updateHeartbeat(StorageReport[] reports,
-      int xceiverCount, int volFailures) {
+  public void updateHeartbeat(StorageReport[] reports, long cacheCapacity,
+      long cacheUsed, int xceiverCount, int volFailures) {
     long totalCapacity = 0;
     long totalCapacity = 0;
     long totalRemaining = 0;
     long totalRemaining = 0;
     long totalBlockPoolUsed = 0;
     long totalBlockPoolUsed = 0;
     long totalDfsUsed = 0;
     long totalDfsUsed = 0;
 
 
+    setCacheCapacity(cacheCapacity);
+    setCacheUsed(cacheUsed);
     setXceiverCount(xceiverCount);
     setXceiverCount(xceiverCount);
     setLastUpdate(Time.now());    
     setLastUpdate(Time.now());    
     this.volumeFailures = volFailures;
     this.volumeFailures = volFailures;
@@ -331,7 +420,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   Iterator<BlockInfo> getBlockIterator(final String storageID) {
   Iterator<BlockInfo> getBlockIterator(final String storageID) {
     return new BlockIterator(getStorageInfo(storageID));
     return new BlockIterator(getStorageInfo(storageID));
   }
   }
-  
+
   /**
   /**
    * Store block replication work.
    * Store block replication work.
    */
    */
@@ -363,7 +452,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       }
       }
     }
     }
   }
   }
-
+  
   /**
   /**
    * The number of work items that are pending to be replicated
    * The number of work items that are pending to be replicated
    */
    */
@@ -380,7 +469,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       return invalidateBlocks.size();
       return invalidateBlocks.size();
     }
     }
   }
   }
-  
+
   public List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
   public List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
     return replicateBlocks.poll(maxTransfers);
     return replicateBlocks.poll(maxTransfers);
   }
   }
@@ -575,5 +664,21 @@ public class DatanodeDescriptor extends DatanodeInfo {
       return storage;
       return storage;
     }
     }
   }
   }
+
+  /**
+   * @return   The time at which we last sent caching directives to this 
+   *           DataNode, in monotonic milliseconds.
+   */
+  public long getLastCachingDirectiveSentTimeMs() {
+    return this.lastCachingDirectiveSentTimeMs;
+  }
+
+  /**
+   * @param time  The time at which we last sent caching directives to this 
+   *              DataNode, in monotonic milliseconds.
+   */
+  public void setLastCachingDirectiveSentTimeMs(long time) {
+    this.lastCachingDirectiveSentTimeMs = time;
+  }
 }
 }
 
 

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

@@ -32,6 +32,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
+import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.HostFileManager;
 import org.apache.hadoop.hdfs.server.namenode.HostFileManager;
 import org.apache.hadoop.hdfs.server.namenode.HostFileManager.Entry;
 import org.apache.hadoop.hdfs.server.namenode.HostFileManager.Entry;
 import org.apache.hadoop.hdfs.server.namenode.HostFileManager.EntrySet;
 import org.apache.hadoop.hdfs.server.namenode.HostFileManager.EntrySet;
@@ -143,6 +145,12 @@ public class DatanodeManager {
   private boolean hasClusterEverBeenMultiRack = false;
   private boolean hasClusterEverBeenMultiRack = false;
 
 
   private final boolean checkIpHostnameInRegistration;
   private final boolean checkIpHostnameInRegistration;
+  /**
+   * Whether we should tell datanodes what to cache in replies to
+   * heartbeat messages.
+   */
+  private boolean shouldSendCachingCommands = false;
+
   /**
   /**
    * The number of datanodes for each software version. This list should change
    * The number of datanodes for each software version. This list should change
    * during rolling upgrades.
    * during rolling upgrades.
@@ -151,6 +159,16 @@ public class DatanodeManager {
   private HashMap<String, Integer> datanodesSoftwareVersions =
   private HashMap<String, Integer> datanodesSoftwareVersions =
     new HashMap<String, Integer>(4, 0.75f);
     new HashMap<String, Integer>(4, 0.75f);
   
   
+  /**
+   * The minimum time between resending caching directives to Datanodes,
+   * in milliseconds.
+   *
+   * Note that when a rescan happens, we will send the new directives
+   * as soon as possible.  This timeout only applies to resending 
+   * directives that we've already sent.
+   */
+  private final long timeBetweenResendingCachingDirectivesMs;
+  
   DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
   DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
       final Configuration conf) throws IOException {
       final Configuration conf) throws IOException {
     this.namesystem = namesystem;
     this.namesystem = namesystem;
@@ -233,6 +251,9 @@ public class DatanodeManager {
         DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY +
         DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY +
         " = '" + ratioUseStaleDataNodesForWrite + "' is invalid. " +
         " = '" + ratioUseStaleDataNodesForWrite + "' is invalid. " +
         "It should be a positive non-zero float value, not greater than 1.0f.");
         "It should be a positive non-zero float value, not greater than 1.0f.");
+    this.timeBetweenResendingCachingDirectivesMs = conf.getLong(
+        DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS,
+        DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS_DEFAULT);
   }
   }
 
 
   private static long getStaleIntervalFromConf(Configuration conf,
   private static long getStaleIntervalFromConf(Configuration conf,
@@ -1203,7 +1224,8 @@ public class DatanodeManager {
   /** Handle heartbeat from datanodes. */
   /** Handle heartbeat from datanodes. */
   public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
   public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
       StorageReport[] reports, final String blockPoolId,
       StorageReport[] reports, final String blockPoolId,
-      int xceiverCount, int maxTransfers, int failedVolumes
+      long cacheCapacity, long cacheUsed, int xceiverCount, 
+      int maxTransfers, int failedVolumes
       ) throws IOException {
       ) throws IOException {
     synchronized (heartbeatManager) {
     synchronized (heartbeatManager) {
       synchronized (datanodeMap) {
       synchronized (datanodeMap) {
@@ -1225,6 +1247,7 @@ public class DatanodeManager {
         }
         }
 
 
         heartbeatManager.updateHeartbeat(nodeinfo, reports,
         heartbeatManager.updateHeartbeat(nodeinfo, reports,
+                                         cacheCapacity, cacheUsed,
                                          xceiverCount, failedVolumes);
                                          xceiverCount, failedVolumes);
 
 
         // If we are in safemode, do not send back any recovery / replication
         // If we are in safemode, do not send back any recovery / replication
@@ -1286,7 +1309,30 @@ public class DatanodeManager {
           cmds.add(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE,
           cmds.add(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE,
               blockPoolId, blks));
               blockPoolId, blks));
         }
         }
-        
+        boolean sendingCachingCommands = false;
+        long nowMs = Time.monotonicNow();
+        if (shouldSendCachingCommands && 
+            ((nowMs - nodeinfo.getLastCachingDirectiveSentTimeMs()) >=
+                timeBetweenResendingCachingDirectivesMs)) {
+          DatanodeCommand pendingCacheCommand =
+              getCacheCommand(nodeinfo.getPendingCached(), nodeinfo,
+                DatanodeProtocol.DNA_CACHE, blockPoolId);
+          if (pendingCacheCommand != null) {
+            cmds.add(pendingCacheCommand);
+            sendingCachingCommands = true;
+          }
+          DatanodeCommand pendingUncacheCommand =
+              getCacheCommand(nodeinfo.getPendingUncached(), nodeinfo,
+                DatanodeProtocol.DNA_UNCACHE, blockPoolId);
+          if (pendingUncacheCommand != null) {
+            cmds.add(pendingUncacheCommand);
+            sendingCachingCommands = true;
+          }
+          if (sendingCachingCommands) {
+            nodeinfo.setLastCachingDirectiveSentTimeMs(nowMs);
+          }
+        }
+
         blockManager.addKeyUpdateCommand(cmds, nodeinfo);
         blockManager.addKeyUpdateCommand(cmds, nodeinfo);
 
 
         // check for balancer bandwidth update
         // check for balancer bandwidth update
@@ -1305,6 +1351,34 @@ public class DatanodeManager {
     return new DatanodeCommand[0];
     return new DatanodeCommand[0];
   }
   }
 
 
+  /**
+   * Convert a CachedBlockList into a DatanodeCommand with a list of blocks.
+   *
+   * @param list       The {@link CachedBlocksList}.  This function 
+   *                   clears the list.
+   * @param datanode   The datanode.
+   * @param action     The action to perform in the command.
+   * @param poolId     The block pool id.
+   * @return           A DatanodeCommand to be sent back to the DN, or null if
+   *                   there is nothing to be done.
+   */
+  private DatanodeCommand getCacheCommand(CachedBlocksList list,
+      DatanodeDescriptor datanode, int action, String poolId) {
+    int length = list.size();
+    if (length == 0) {
+      return null;
+    }
+    // Read the existing cache commands.
+    long[] blockIds = new long[length];
+    int i = 0;
+    for (Iterator<CachedBlock> iter = list.iterator();
+            iter.hasNext(); ) {
+      CachedBlock cachedBlock = iter.next();
+      blockIds[i++] = cachedBlock.getBlockId();
+    }
+    return new BlockIdCommand(action, poolId, blockIds);
+  }
+
   /**
   /**
    * Tell all datanodes to use a new, non-persistent bandwidth value for
    * Tell all datanodes to use a new, non-persistent bandwidth value for
    * dfs.balance.bandwidthPerSec.
    * dfs.balance.bandwidthPerSec.
@@ -1351,9 +1425,32 @@ public class DatanodeManager {
     }
     }
   }
   }
 
 
+  /**
+   * Reset the lastCachingDirectiveSentTimeMs field of all the DataNodes we
+   * know about.
+   */
+  public void resetLastCachingDirectiveSentTime() {
+    synchronized (datanodeMap) {
+      for (DatanodeDescriptor dn : datanodeMap.values()) {
+        dn.setLastCachingDirectiveSentTimeMs(0L);
+      }
+    }
+  }
+
   @Override
   @Override
   public String toString() {
   public String toString() {
     return getClass().getSimpleName() + ": " + host2DatanodeMap;
     return getClass().getSimpleName() + ": " + host2DatanodeMap;
   }
   }
+
+  public void clearPendingCachingCommands() {
+    for (DatanodeDescriptor dn : datanodeMap.values()) {
+      dn.getPendingCached().clear();
+      dn.getPendingUncached().clear();
+    }
+  }
+
+  public void setShouldSendCachingCommands(boolean shouldSendCachingCommands) {
+    this.shouldSendCachingCommands = shouldSendCachingCommands;
+  }
 }
 }
 
 

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java

@@ -42,6 +42,12 @@ public interface DatanodeStatistics {
 
 
   /** @return the percentage of the block pool used space over the total capacity. */
   /** @return the percentage of the block pool used space over the total capacity. */
   public float getPercentBlockPoolUsed();
   public float getPercentBlockPoolUsed();
+  
+  /** @return the total cache capacity of all DataNodes */
+  public long getCacheCapacity();
+
+  /** @return the total cache used by all DataNodes */
+  public long getCacheUsed();
 
 
   /** @return the xceiver count */
   /** @return the xceiver count */
   public int getXceiverCount();
   public int getXceiverCount();

+ 22 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java

@@ -149,6 +149,17 @@ class HeartbeatManager implements DatanodeStatistics {
   public synchronized int getXceiverCount() {
   public synchronized int getXceiverCount() {
     return stats.xceiverCount;
     return stats.xceiverCount;
   }
   }
+  
+  @Override
+  public synchronized long getCacheCapacity() {
+    return stats.cacheCapacity;
+  }
+
+  @Override
+  public synchronized long getCacheUsed() {
+    return stats.cacheUsed;
+  }
+  
 
 
   @Override
   @Override
   public synchronized long[] getStats() {
   public synchronized long[] getStats() {
@@ -171,7 +182,7 @@ class HeartbeatManager implements DatanodeStatistics {
       addDatanode(d);
       addDatanode(d);
 
 
       //update its timestamp
       //update its timestamp
-      d.updateHeartbeat(StorageReport.EMPTY_ARRAY, 0, 0);
+      d.updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
     }
     }
   }
   }
 
 
@@ -193,9 +204,11 @@ class HeartbeatManager implements DatanodeStatistics {
   }
   }
 
 
   synchronized void updateHeartbeat(final DatanodeDescriptor node,
   synchronized void updateHeartbeat(final DatanodeDescriptor node,
-      StorageReport[] reports, int xceiverCount, int failedVolumes) {
+      StorageReport[] reports, long cacheCapacity, long cacheUsed,
+      int xceiverCount, int failedVolumes) {
     stats.subtract(node);
     stats.subtract(node);
-    node.updateHeartbeat(reports, xceiverCount, failedVolumes);
+    node.updateHeartbeat(reports, cacheCapacity, cacheUsed,
+        xceiverCount, failedVolumes);
     stats.add(node);
     stats.add(node);
   }
   }
 
 
@@ -307,6 +320,8 @@ class HeartbeatManager implements DatanodeStatistics {
     private long capacityRemaining = 0L;
     private long capacityRemaining = 0L;
     private long blockPoolUsed = 0L;
     private long blockPoolUsed = 0L;
     private int xceiverCount = 0;
     private int xceiverCount = 0;
+    private long cacheCapacity = 0L;
+    private long cacheUsed = 0L;
 
 
     private int expiredHeartbeats = 0;
     private int expiredHeartbeats = 0;
 
 
@@ -320,6 +335,8 @@ class HeartbeatManager implements DatanodeStatistics {
       } else {
       } else {
         capacityTotal += node.getDfsUsed();
         capacityTotal += node.getDfsUsed();
       }
       }
+      cacheCapacity += node.getCacheCapacity();
+      cacheUsed += node.getCacheUsed();
     }
     }
 
 
     private void subtract(final DatanodeDescriptor node) {
     private void subtract(final DatanodeDescriptor node) {
@@ -332,6 +349,8 @@ class HeartbeatManager implements DatanodeStatistics {
       } else {
       } else {
         capacityTotal -= node.getDfsUsed();
         capacityTotal -= node.getDfsUsed();
       }
       }
+      cacheCapacity -= node.getCacheCapacity();
+      cacheUsed -= node.getCacheUsed();
     }
     }
     
     
     /** Increment expired heartbeat counter. */
     /** Increment expired heartbeat counter. */

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

@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -538,6 +539,21 @@ class BPOfferService {
   }
   }
   }
   }
 
 
+  private String blockIdArrayToString(long ids[]) {
+    long maxNumberOfBlocksToLog = dn.getMaxNumberOfBlocksToLog();
+    StringBuilder bld = new StringBuilder();
+    String prefix = "";
+    for (int i = 0; i < ids.length; i++) {
+      if (i >= maxNumberOfBlocksToLog) {
+        bld.append("...");
+        break;
+      }
+      bld.append(prefix).append(ids[i]);
+      prefix = ", ";
+    }
+    return bld.toString();
+  }
+
   /**
   /**
    * This method should handle all commands from Active namenode except
    * This method should handle all commands from Active namenode except
    * DNA_REGISTER which should be handled earlier itself.
    * DNA_REGISTER which should be handled earlier itself.
@@ -550,6 +566,8 @@ class BPOfferService {
       BPServiceActor actor) throws IOException {
       BPServiceActor actor) throws IOException {
     final BlockCommand bcmd = 
     final BlockCommand bcmd = 
       cmd instanceof BlockCommand? (BlockCommand)cmd: null;
       cmd instanceof BlockCommand? (BlockCommand)cmd: null;
+    final BlockIdCommand blockIdCmd = 
+      cmd instanceof BlockIdCommand ? (BlockIdCommand)cmd: null;
 
 
     switch(cmd.getAction()) {
     switch(cmd.getAction()) {
     case DatanodeProtocol.DNA_TRANSFER:
     case DatanodeProtocol.DNA_TRANSFER:
@@ -575,6 +593,20 @@ class BPOfferService {
       }
       }
       dn.metrics.incrBlocksRemoved(toDelete.length);
       dn.metrics.incrBlocksRemoved(toDelete.length);
       break;
       break;
+    case DatanodeProtocol.DNA_CACHE:
+      LOG.info("DatanodeCommand action: DNA_CACHE for " +
+        blockIdCmd.getBlockPoolId() + " of [" +
+          blockIdArrayToString(blockIdCmd.getBlockIds()) + "]");
+      dn.getFSDataset().cache(blockIdCmd.getBlockPoolId(), blockIdCmd.getBlockIds());
+      dn.metrics.incrBlocksCached(blockIdCmd.getBlockIds().length);
+      break;
+    case DatanodeProtocol.DNA_UNCACHE:
+      LOG.info("DatanodeCommand action: DNA_UNCACHE for " +
+        blockIdCmd.getBlockPoolId() + " of [" +
+          blockIdArrayToString(blockIdCmd.getBlockIds()) + "]");
+      dn.getFSDataset().uncache(blockIdCmd.getBlockPoolId(), blockIdCmd.getBlockIds());
+      dn.metrics.incrBlocksUncached(blockIdCmd.getBlockIds().length);
+      break;
     case DatanodeProtocol.DNA_SHUTDOWN:
     case DatanodeProtocol.DNA_SHUTDOWN:
       // TODO: DNA_SHUTDOWN appears to be unused - the NN never sends this command
       // TODO: DNA_SHUTDOWN appears to be unused - the NN never sends this command
       // See HDFS-2987.
       // See HDFS-2987.
@@ -639,6 +671,8 @@ class BPOfferService {
     case DatanodeProtocol.DNA_FINALIZE:
     case DatanodeProtocol.DNA_FINALIZE:
     case DatanodeProtocol.DNA_RECOVERBLOCK:
     case DatanodeProtocol.DNA_RECOVERBLOCK:
     case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
     case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
+    case DatanodeProtocol.DNA_CACHE:
+    case DatanodeProtocol.DNA_UNCACHE:
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
       break;
       break;
     default:
     default:

+ 43 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -24,6 +24,7 @@ import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
 import java.net.SocketTimeoutException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -86,6 +87,8 @@ class BPServiceActor implements Runnable {
 
 
   boolean resetBlockReportTime = true;
   boolean resetBlockReportTime = true;
 
 
+  volatile long lastCacheReport = 0;
+
   Thread bpThread;
   Thread bpThread;
   DatanodeProtocolClientSideTranslatorPB bpNamenode;
   DatanodeProtocolClientSideTranslatorPB bpNamenode;
   private volatile long lastHeartbeat = 0;
   private volatile long lastHeartbeat = 0;
@@ -503,6 +506,35 @@ class BPServiceActor implements Runnable {
     return cmd;
     return cmd;
   }
   }
   
   
+  DatanodeCommand cacheReport() throws IOException {
+    // If caching is disabled, do not send a cache report
+    if (dn.getFSDataset().getCacheCapacity() == 0) {
+      return null;
+    }
+    // send cache report if timer has expired.
+    DatanodeCommand cmd = null;
+    long startTime = Time.monotonicNow();
+    if (startTime - lastCacheReport > dnConf.cacheReportInterval) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Sending cacheReport from service actor: " + this);
+      }
+      lastCacheReport = startTime;
+
+      String bpid = bpos.getBlockPoolId();
+      List<Long> blockIds = dn.getFSDataset().getCacheReport(bpid);
+      long createTime = Time.monotonicNow();
+
+      cmd = bpNamenode.cacheReport(bpRegistration, bpid, blockIds);
+      long sendTime = Time.monotonicNow();
+      long createCost = createTime - startTime;
+      long sendCost = sendTime - createTime;
+      dn.getMetrics().addCacheReport(sendCost);
+      LOG.debug("CacheReport of " + blockIds.size()
+          + " block(s) took " + createCost + " msec to generate and "
+          + sendCost + " msecs for RPC and NN processing");
+    }
+    return cmd;
+  }
   
   
   HeartbeatResponse sendHeartBeat() throws IOException {
   HeartbeatResponse sendHeartBeat() throws IOException {
     StorageReport[] reports =
     StorageReport[] reports =
@@ -514,6 +546,8 @@ class BPServiceActor implements Runnable {
 
 
     return bpNamenode.sendHeartbeat(bpRegistration,
     return bpNamenode.sendHeartbeat(bpRegistration,
         reports,
         reports,
+        dn.getFSDataset().getCacheCapacity(),
+        dn.getFSDataset().getCacheUsed(),
         dn.getXmitsInProgress(),
         dn.getXmitsInProgress(),
         dn.getXceiverCount(),
         dn.getXceiverCount(),
         dn.getFSDataset().getNumFailedVolumes());
         dn.getFSDataset().getNumFailedVolumes());
@@ -567,11 +601,12 @@ class BPServiceActor implements Runnable {
    * forever calling remote NameNode functions.
    * forever calling remote NameNode functions.
    */
    */
   private void offerService() throws Exception {
   private void offerService() throws Exception {
-    LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
-        + dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
-        + dnConf.blockReportInterval + "msec" + " Initial delay: "
-        + dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
-        + dnConf.heartBeatInterval);
+    LOG.info("For namenode " + nnAddr + " using"
+        + " DELETEREPORT_INTERVAL of " + dnConf.deleteReportInterval + " msec "
+        + " BLOCKREPORT_INTERVAL of " + dnConf.blockReportInterval + "msec"
+        + " CACHEREPORT_INTERVAL of " + dnConf.cacheReportInterval + "msec"
+        + " Initial delay: " + dnConf.initialBlockReportDelay + "msec"
+        + "; heartBeatInterval=" + dnConf.heartBeatInterval);
 
 
     //
     //
     // Now loop for a long time....
     // Now loop for a long time....
@@ -627,6 +662,9 @@ class BPServiceActor implements Runnable {
         DatanodeCommand cmd = blockReport();
         DatanodeCommand cmd = blockReport();
         processCommand(new DatanodeCommand[]{ cmd });
         processCommand(new DatanodeCommand[]{ cmd });
 
 
+        cmd = cacheReport();
+        processCommand(new DatanodeCommand[]{ cmd });
+
         // Now safe to start scanning the block pool.
         // Now safe to start scanning the block pool.
         // If it has already been started, this is a no-op.
         // If it has already been started, this is a no-op.
         if (dn.blockScanner != null) {
         if (dn.blockScanner != null) {

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

@@ -664,8 +664,9 @@ class BlockReceiver implements Closeable {
         //                     
         //                     
         long dropPos = lastCacheManagementOffset - CACHE_DROP_LAG_BYTES;
         long dropPos = lastCacheManagementOffset - CACHE_DROP_LAG_BYTES;
         if (dropPos > 0 && dropCacheBehindWrites) {
         if (dropPos > 0 && dropCacheBehindWrites) {
-          NativeIO.POSIX.posixFadviseIfPossible(block.getBlockName(),
-              outFd, 0, dropPos, NativeIO.POSIX.POSIX_FADV_DONTNEED);
+          NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+              block.getBlockName(), outFd, 0, dropPos,
+              NativeIO.POSIX.POSIX_FADV_DONTNEED);
         }
         }
         lastCacheManagementOffset = offsetInBlock;
         lastCacheManagementOffset = offsetInBlock;
       }
       }

+ 9 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -375,8 +375,9 @@ class BlockSender implements java.io.Closeable {
         ((dropCacheBehindAllReads) ||
         ((dropCacheBehindAllReads) ||
          (dropCacheBehindLargeReads && isLongRead()))) {
          (dropCacheBehindLargeReads && isLongRead()))) {
       try {
       try {
-        NativeIO.POSIX.posixFadviseIfPossible(block.getBlockName(),
-            blockInFd, lastCacheDropOffset, offset - lastCacheDropOffset,
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+            block.getBlockName(), blockInFd, lastCacheDropOffset,
+            offset - lastCacheDropOffset,
             NativeIO.POSIX.POSIX_FADV_DONTNEED);
             NativeIO.POSIX.POSIX_FADV_DONTNEED);
       } catch (Exception e) {
       } catch (Exception e) {
         LOG.warn("Unable to drop cache on file close", e);
         LOG.warn("Unable to drop cache on file close", e);
@@ -674,8 +675,9 @@ class BlockSender implements java.io.Closeable {
 
 
     if (isLongRead() && blockInFd != null) {
     if (isLongRead() && blockInFd != null) {
       // Advise that this file descriptor will be accessed sequentially.
       // Advise that this file descriptor will be accessed sequentially.
-      NativeIO.POSIX.posixFadviseIfPossible(block.getBlockName(),
-          blockInFd, 0, 0, NativeIO.POSIX.POSIX_FADV_SEQUENTIAL);
+      NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+          block.getBlockName(), blockInFd, 0, 0,
+          NativeIO.POSIX.POSIX_FADV_SEQUENTIAL);
     }
     }
     
     
     // Trigger readahead of beginning of file if configured.
     // Trigger readahead of beginning of file if configured.
@@ -761,9 +763,9 @@ class BlockSender implements java.io.Closeable {
       long nextCacheDropOffset = lastCacheDropOffset + CACHE_DROP_INTERVAL_BYTES;
       long nextCacheDropOffset = lastCacheDropOffset + CACHE_DROP_INTERVAL_BYTES;
       if (offset >= nextCacheDropOffset) {
       if (offset >= nextCacheDropOffset) {
         long dropLength = offset - lastCacheDropOffset;
         long dropLength = offset - lastCacheDropOffset;
-        NativeIO.POSIX.posixFadviseIfPossible(block.getBlockName(),
-            blockInFd, lastCacheDropOffset, dropLength,
-            NativeIO.POSIX.POSIX_FADV_DONTNEED);
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+            block.getBlockName(), blockInFd, lastCacheDropOffset,
+            dropLength, NativeIO.POSIX.POSIX_FADV_DONTNEED);
         lastCacheDropOffset = offset;
         lastCacheDropOffset = offset;
       }
       }
     }
     }

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

@@ -18,13 +18,18 @@
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
@@ -39,6 +44,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_SUPPORTED_NA
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -66,6 +72,7 @@ public class DNConf {
   final long blockReportInterval;
   final long blockReportInterval;
   final long deleteReportInterval;
   final long deleteReportInterval;
   final long initialBlockReportDelay;
   final long initialBlockReportDelay;
+  final long cacheReportInterval;
   final int writePacketSize;
   final int writePacketSize;
   
   
   final String minimumNameNodeVersion;
   final String minimumNameNodeVersion;
@@ -73,6 +80,8 @@ public class DNConf {
   
   
   final long xceiverStopTimeout;
   final long xceiverStopTimeout;
 
 
+  final long maxLockedMemory;
+
   public DNConf(Configuration conf) {
   public DNConf(Configuration conf) {
     socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
     socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
         HdfsServerConstants.READ_TIMEOUT);
         HdfsServerConstants.READ_TIMEOUT);
@@ -107,7 +116,9 @@ public class DNConf {
         DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME,
         DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME,
         DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
         DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
     this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
     this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
-    DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
+        DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
+    this.cacheReportInterval = conf.getLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
+        DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT);
     
     
     long initBRDelay = conf.getLong(
     long initBRDelay = conf.getLong(
         DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
         DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
@@ -137,6 +148,10 @@ public class DNConf {
     this.xceiverStopTimeout = conf.getLong(
     this.xceiverStopTimeout = conf.getLong(
         DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY,
         DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY,
         DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT);
         DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT);
+
+    this.maxLockedMemory = conf.getLong(
+        DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
+        DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT);
   }
   }
   
   
   // We get minimumNameNodeVersion via a method so it can be mocked out in tests.
   // We get minimumNameNodeVersion via a method so it can be mocked out in tests.
@@ -147,4 +162,8 @@ public class DNConf {
   public long getXceiverStopTimeout() {
   public long getXceiverStopTimeout() {
     return xceiverStopTimeout;
     return xceiverStopTimeout;
   }
   }
+
+  public long getMaxLockedMemory() {
+    return maxLockedMemory;
+  }
 }
 }

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -124,6 +124,7 @@ import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.ReadaheadPool;
 import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
@@ -264,6 +265,7 @@ public class DataNode extends Configured
   private SecureResources secureResources = null;
   private SecureResources secureResources = null;
   private List<StorageLocation> dataDirs;
   private List<StorageLocation> dataDirs;
   private Configuration conf;
   private Configuration conf;
+  private final long maxNumberOfBlocksToLog;
 
 
   private final List<String> usersWithLocalPathAccess;
   private final List<String> usersWithLocalPathAccess;
   private boolean connectToDnViaHostname;
   private boolean connectToDnViaHostname;
@@ -279,6 +281,8 @@ public class DataNode extends Configured
            final List<StorageLocation> dataDirs,
            final List<StorageLocation> dataDirs,
            final SecureResources resources) throws IOException {
            final SecureResources resources) throws IOException {
     super(conf);
     super(conf);
+    this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
+        DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT);
 
 
     this.usersWithLocalPathAccess = Arrays.asList(
     this.usersWithLocalPathAccess = Arrays.asList(
         conf.getTrimmedStrings(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY));
         conf.getTrimmedStrings(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY));
@@ -737,6 +741,27 @@ public class DataNode extends Configured
     this.conf = conf;
     this.conf = conf;
     this.dnConf = new DNConf(conf);
     this.dnConf = new DNConf(conf);
 
 
+    if (dnConf.maxLockedMemory > 0) {
+      if (!NativeIO.POSIX.getCacheManipulator().verifyCanMlock()) {
+        throw new RuntimeException(String.format(
+            "Cannot start datanode because the configured max locked memory" +
+            " size (%s) is greater than zero and native code is not available.",
+            DFS_DATANODE_MAX_LOCKED_MEMORY_KEY));
+      }
+      long ulimit = NativeIO.POSIX.getCacheManipulator().getMemlockLimit();
+      if (dnConf.maxLockedMemory > ulimit) {
+      throw new RuntimeException(String.format(
+          "Cannot start datanode because the configured max locked memory" +
+          " size (%s) of %d bytes is more than the datanode's available" +
+          " RLIMIT_MEMLOCK ulimit of %d bytes.",
+          DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
+          dnConf.maxLockedMemory,
+          ulimit));
+      }
+    }
+    LOG.info("Starting DataNode with maxLockedMemory = " +
+        dnConf.maxLockedMemory);
+
     storage = new DataStorage();
     storage = new DataStorage();
     
     
     // global DN settings
     // global DN settings
@@ -1075,6 +1100,10 @@ public class DataNode extends Configured
     }
     }
   }
   }
 
 
+  public long getMaxNumberOfBlocksToLog() {
+    return maxNumberOfBlocksToLog;
+  }
+
   @Override
   @Override
   public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
   public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
       Token<BlockTokenIdentifier> token) throws IOException {
       Token<BlockTokenIdentifier> token) throws IOException {

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java

@@ -273,6 +273,14 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    */
    */
   public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
   public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
 
 
+  /**
+   * Returns the cache report - the full list of cached block IDs of a
+   * block pool.
+   * @param   bpid Block Pool Id
+   * @return  the cache report - the full list of cached block IDs.
+   */
+  public List<Long> getCacheReport(String bpid);
+
   /** Does the dataset contain the block? */
   /** Does the dataset contain the block? */
   public boolean contains(ExtendedBlock block);
   public boolean contains(ExtendedBlock block);
 
 
@@ -298,6 +306,20 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    */
    */
   public void invalidate(String bpid, Block invalidBlks[]) throws IOException;
   public void invalidate(String bpid, Block invalidBlks[]) throws IOException;
 
 
+  /**
+   * Caches the specified blocks
+   * @param bpid Block pool id
+   * @param blockIds - block ids to cache
+   */
+  public void cache(String bpid, long[] blockIds);
+
+  /**
+   * Uncaches the specified blocks
+   * @param bpid Block pool id
+   * @param blockIds - blocks ids to uncache
+   */
+  public void uncache(String bpid, long[] blockIds);
+
     /**
     /**
      * Check if all the data directories are healthy
      * Check if all the data directories are healthy
      * @throws DiskErrorException
      * @throws DiskErrorException

+ 506 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java

@@ -0,0 +1,506 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.io.nativeio.NativeIO;
+
+/**
+ * Manages caching for an FsDatasetImpl by using the mmap(2) and mlock(2)
+ * system calls to lock blocks into memory. Block checksums are verified upon
+ * entry into the cache.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class FsDatasetCache {
+  /**
+   * Keys which identify MappableBlocks.
+   */
+  private static final class Key {
+    /**
+     * Block id.
+     */
+    final long id;
+
+    /**
+     * Block pool id.
+     */
+    final String bpid;
+
+    Key(long id, String bpid) {
+      this.id = id;
+      this.bpid = bpid;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (o == null) {
+        return false;
+      }
+      if (!(o.getClass() == getClass())) {
+        return false;
+      }
+      Key other = (Key)o;
+      return ((other.id == this.id) && (other.bpid.equals(this.bpid)));
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().append(id).append(bpid).hashCode();
+    }
+  };
+
+  /**
+   * MappableBlocks that we know about.
+   */
+  private static final class Value {
+    final State state;
+    final MappableBlock mappableBlock;
+
+    Value(MappableBlock mappableBlock, State state) {
+      this.mappableBlock = mappableBlock;
+      this.state = state;
+    }
+  }
+
+  private enum State {
+    /**
+     * The MappableBlock is in the process of being cached.
+     */
+    CACHING,
+
+    /**
+     * The MappableBlock was in the process of being cached, but it was
+     * cancelled.  Only the FsDatasetCache#WorkerTask can remove cancelled
+     * MappableBlock objects.
+     */
+    CACHING_CANCELLED,
+
+    /**
+     * The MappableBlock is in the cache.
+     */
+    CACHED,
+
+    /**
+     * The MappableBlock is in the process of uncaching.
+     */
+    UNCACHING;
+
+    /**
+     * Whether we should advertise this block as cached to the NameNode and
+     * clients.
+     */
+    public boolean shouldAdvertise() {
+      return (this == CACHED);
+    }
+  }
+
+  private static final Log LOG = LogFactory.getLog(FsDatasetCache.class);
+
+  /**
+   * Stores MappableBlock objects and the states they're in.
+   */
+  private final HashMap<Key, Value> mappableBlockMap = new HashMap<Key, Value>();
+
+  private final AtomicLong numBlocksCached = new AtomicLong(0);
+
+  private final FsDatasetImpl dataset;
+
+  private final ThreadPoolExecutor uncachingExecutor;
+
+  /**
+   * The approximate amount of cache space in use.
+   *
+   * This number is an overestimate, counting bytes that will be used only
+   * if pending caching operations succeed.  It does not take into account
+   * pending uncaching operations.
+   *
+   * This overestimate is more useful to the NameNode than an underestimate,
+   * since we don't want the NameNode to assign us more replicas than
+   * we can cache, because of the current batch of operations.
+   */
+  private final UsedBytesCount usedBytesCount;
+
+  public static class PageRounder {
+    private final long osPageSize =
+        NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize();
+
+    /**
+     * Round up a number to the operating system page size.
+     */
+    public long round(long count) {
+      long newCount = 
+          (count + (osPageSize - 1)) / osPageSize;
+      return newCount * osPageSize;
+    }
+  }
+
+  private class UsedBytesCount {
+    private final AtomicLong usedBytes = new AtomicLong(0);
+    
+    private PageRounder rounder = new PageRounder();
+
+    /**
+     * Try to reserve more bytes.
+     *
+     * @param count    The number of bytes to add.  We will round this
+     *                 up to the page size.
+     *
+     * @return         The new number of usedBytes if we succeeded;
+     *                 -1 if we failed.
+     */
+    long reserve(long count) {
+      count = rounder.round(count);
+      while (true) {
+        long cur = usedBytes.get();
+        long next = cur + count;
+        if (next > maxBytes) {
+          return -1;
+        }
+        if (usedBytes.compareAndSet(cur, next)) {
+          return next;
+        }
+      }
+    }
+    
+    /**
+     * Release some bytes that we're using.
+     *
+     * @param count    The number of bytes to release.  We will round this
+     *                 up to the page size.
+     *
+     * @return         The new number of usedBytes.
+     */
+    long release(long count) {
+      count = rounder.round(count);
+      return usedBytes.addAndGet(-count);
+    }
+    
+    long get() {
+      return usedBytes.get();
+    }
+  }
+
+  /**
+   * The total cache capacity in bytes.
+   */
+  private final long maxBytes;
+
+  /**
+   * Number of cache commands that could not be completed successfully
+   */
+  AtomicLong numBlocksFailedToCache = new AtomicLong(0);
+  /**
+   * Number of uncache commands that could not be completed successfully
+   */
+  AtomicLong numBlocksFailedToUncache = new AtomicLong(0);
+
+  public FsDatasetCache(FsDatasetImpl dataset) {
+    this.dataset = dataset;
+    this.maxBytes = dataset.datanode.getDnConf().getMaxLockedMemory();
+    ThreadFactory workerFactory = new ThreadFactoryBuilder()
+        .setDaemon(true)
+        .setNameFormat("FsDatasetCache-%d-" + dataset.toString())
+        .build();
+    this.usedBytesCount = new UsedBytesCount();
+    this.uncachingExecutor = new ThreadPoolExecutor(
+            0, 1,
+            60, TimeUnit.SECONDS,
+            new LinkedBlockingQueue<Runnable>(),
+            workerFactory);
+    this.uncachingExecutor.allowCoreThreadTimeOut(true);
+  }
+
+  /**
+   * @return List of cached blocks suitable for translation into a
+   * {@link BlockListAsLongs} for a cache report.
+   */
+  synchronized List<Long> getCachedBlocks(String bpid) {
+    List<Long> blocks = new ArrayList<Long>();
+    for (Iterator<Entry<Key, Value>> iter =
+        mappableBlockMap.entrySet().iterator(); iter.hasNext(); ) {
+      Entry<Key, Value> entry = iter.next();
+      if (entry.getKey().bpid.equals(bpid)) {
+        if (entry.getValue().state.shouldAdvertise()) {
+          blocks.add(entry.getKey().id);
+        }
+      }
+    }
+    return blocks;
+  }
+
+  /**
+   * Attempt to begin caching a block.
+   */
+  synchronized void cacheBlock(long blockId, String bpid,
+      String blockFileName, long length, long genstamp,
+      Executor volumeExecutor) {
+    Key key = new Key(blockId, bpid);
+    Value prevValue = mappableBlockMap.get(key);
+    if (prevValue != null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Block with id " + blockId + ", pool " + bpid +
+            " already exists in the FsDatasetCache with state " +
+            prevValue.state);
+      }
+      numBlocksFailedToCache.incrementAndGet();
+      return;
+    }
+    mappableBlockMap.put(key, new Value(null, State.CACHING));
+    volumeExecutor.execute(
+        new CachingTask(key, blockFileName, length, genstamp));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Initiating caching for Block with id " + blockId +
+          ", pool " + bpid);
+    }
+  }
+
+  synchronized void uncacheBlock(String bpid, long blockId) {
+    Key key = new Key(blockId, bpid);
+    Value prevValue = mappableBlockMap.get(key);
+
+    if (prevValue == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Block with id " + blockId + ", pool " + bpid + " " +
+            "does not need to be uncached, because it is not currently " +
+            "in the mappableBlockMap.");
+      }
+      numBlocksFailedToUncache.incrementAndGet();
+      return;
+    }
+    switch (prevValue.state) {
+    case CACHING:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Cancelling caching for block with id " + blockId +
+            ", pool " + bpid + ".");
+      }
+      mappableBlockMap.put(key,
+          new Value(prevValue.mappableBlock, State.CACHING_CANCELLED));
+      break;
+    case CACHED:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Block with id " + blockId + ", pool " + bpid + " " +
+            "has been scheduled for uncaching.");
+      }
+      mappableBlockMap.put(key,
+          new Value(prevValue.mappableBlock, State.UNCACHING));
+      uncachingExecutor.execute(new UncachingTask(key));
+      break;
+    default:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Block with id " + blockId + ", pool " + bpid + " " +
+            "does not need to be uncached, because it is " +
+            "in state " + prevValue.state + ".");
+      }
+      numBlocksFailedToUncache.incrementAndGet();
+      break;
+    }
+  }
+
+  /**
+   * Background worker that mmaps, mlocks, and checksums a block
+   */
+  private class CachingTask implements Runnable {
+    private final Key key; 
+    private final String blockFileName;
+    private final long length;
+    private final long genstamp;
+
+    CachingTask(Key key, String blockFileName, long length, long genstamp) {
+      this.key = key;
+      this.blockFileName = blockFileName;
+      this.length = length;
+      this.genstamp = genstamp;
+    }
+
+    @Override
+    public void run() {
+      boolean success = false;
+      FileInputStream blockIn = null, metaIn = null;
+      MappableBlock mappableBlock = null;
+      ExtendedBlock extBlk =
+          new ExtendedBlock(key.bpid, key.id, length, genstamp);
+      long newUsedBytes = usedBytesCount.reserve(length);
+      if (newUsedBytes < 0) {
+        LOG.warn("Failed to cache block id " + key.id + ", pool " + key.bpid +
+            ": could not reserve " + length + " more bytes in the " +
+            "cache: " + DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY +
+            " of " + maxBytes + " exceeded.");
+        numBlocksFailedToCache.incrementAndGet();
+        return;
+      }
+      try {
+        try {
+          blockIn = (FileInputStream)dataset.getBlockInputStream(extBlk, 0);
+          metaIn = (FileInputStream)dataset.getMetaDataInputStream(extBlk)
+              .getWrappedStream();
+        } catch (ClassCastException e) {
+          LOG.warn("Failed to cache block with id " + key.id + ", pool " +
+                key.bpid + ": Underlying blocks are not backed by files.", e);
+          return;
+        } catch (FileNotFoundException e) {
+          LOG.info("Failed to cache block with id " + key.id + ", pool " +
+                key.bpid + ": failed to find backing files.");
+          return;
+        } catch (IOException e) {
+          LOG.warn("Failed to cache block with id " + key.id + ", pool " +
+                key.bpid + ": failed to open file", e);
+          return;
+        }
+        try {
+          mappableBlock = MappableBlock.
+              load(length, blockIn, metaIn, blockFileName);
+        } catch (ChecksumException e) {
+          // Exception message is bogus since this wasn't caused by a file read
+          LOG.warn("Failed to cache block " + key.id + " in " + key.bpid + ": " +
+                   "checksum verification failed.");
+          return;
+        } catch (IOException e) {
+          LOG.warn("Failed to cache block " + key.id + " in " + key.bpid, e);
+          return;
+        }
+        synchronized (FsDatasetCache.this) {
+          Value value = mappableBlockMap.get(key);
+          Preconditions.checkNotNull(value);
+          Preconditions.checkState(value.state == State.CACHING ||
+                                   value.state == State.CACHING_CANCELLED);
+          if (value.state == State.CACHING_CANCELLED) {
+            mappableBlockMap.remove(key);
+            LOG.warn("Caching of block " + key.id + " in " + key.bpid +
+                " was cancelled.");
+            return;
+          }
+          mappableBlockMap.put(key, new Value(mappableBlock, State.CACHED));
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Successfully cached block " + key.id + " in " + key.bpid +
+              ".  We are now caching " + newUsedBytes + " bytes in total.");
+        }
+        numBlocksCached.addAndGet(1);
+        success = true;
+      } finally {
+        if (!success) {
+          newUsedBytes = usedBytesCount.release(length);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Caching of block " + key.id + " in " +
+              key.bpid + " was aborted.  We are now caching only " +
+              newUsedBytes + " + bytes in total.");
+          }
+          IOUtils.closeQuietly(blockIn);
+          IOUtils.closeQuietly(metaIn);
+          if (mappableBlock != null) {
+            mappableBlock.close();
+          }
+          numBlocksFailedToCache.incrementAndGet();
+
+          synchronized (FsDatasetCache.this) {
+            mappableBlockMap.remove(key);
+          }
+        }
+      }
+    }
+  }
+
+  private class UncachingTask implements Runnable {
+    private final Key key; 
+
+    UncachingTask(Key key) {
+      this.key = key;
+    }
+
+    @Override
+    public void run() {
+      Value value;
+      
+      synchronized (FsDatasetCache.this) {
+        value = mappableBlockMap.get(key);
+      }
+      Preconditions.checkNotNull(value);
+      Preconditions.checkArgument(value.state == State.UNCACHING);
+      // TODO: we will eventually need to do revocation here if any clients
+      // are reading via mmap with checksums enabled.  See HDFS-5182.
+      IOUtils.closeQuietly(value.mappableBlock);
+      synchronized (FsDatasetCache.this) {
+        mappableBlockMap.remove(key);
+      }
+      long newUsedBytes =
+          usedBytesCount.release(value.mappableBlock.getLength());
+      numBlocksCached.addAndGet(-1);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Uncaching of block " + key.id + " in " + key.bpid +
+            " completed.  usedBytes = " + newUsedBytes);
+      }
+    }
+  }
+
+  // Stats related methods for FSDatasetMBean
+
+  /**
+   * Get the approximate amount of cache space used.
+   */
+  public long getCacheUsed() {
+    return usedBytesCount.get();
+  }
+
+  /**
+   * Get the maximum amount of bytes we can cache.  This is a constant.
+   */
+  public long getCacheCapacity() {
+    return maxBytes;
+  }
+
+  public long getNumBlocksFailedToCache() {
+    return numBlocksFailedToCache.get();
+  }
+
+  public long getNumBlocksFailedToUncache() {
+    return numBlocksFailedToUncache.get();
+  }
+
+  public long getNumBlocksCached() {
+    return numBlocksCached.get();
+  }
+}

+ 105 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -32,6 +32,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.concurrent.Executor;
 
 
 import javax.management.NotCompliantMBeanException;
 import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
 import javax.management.ObjectName;
@@ -194,6 +195,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   final DataNode datanode;
   final DataNode datanode;
   final FsVolumeList volumes;
   final FsVolumeList volumes;
   final FsDatasetAsyncDiskService asyncDiskService;
   final FsDatasetAsyncDiskService asyncDiskService;
+  final FsDatasetCache cacheManager;
   private final int validVolsRequired;
   private final int validVolsRequired;
 
 
   final ReplicaMap volumeMap;
   final ReplicaMap volumeMap;
@@ -258,6 +260,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       roots[idx] = storage.getStorageDir(idx).getCurrentDir();
       roots[idx] = storage.getStorageDir(idx).getCurrentDir();
     }
     }
     asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots);
     asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots);
+    cacheManager = new FsDatasetCache(this);
     registerMBean(datanode.getDatanodeUuid());
     registerMBean(datanode.getDatanodeUuid());
   }
   }
 
 
@@ -327,6 +330,31 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     return volumes.numberOfFailedVolumes();
     return volumes.numberOfFailedVolumes();
   }
   }
 
 
+  @Override // FSDatasetMBean
+  public long getCacheUsed() {
+    return cacheManager.getCacheUsed();
+  }
+
+  @Override // FSDatasetMBean
+  public long getCacheCapacity() {
+    return cacheManager.getCacheCapacity();
+  }
+
+  @Override // FSDatasetMBean
+  public long getNumBlocksFailedToCache() {
+    return cacheManager.getNumBlocksFailedToCache();
+  }
+
+  @Override // FSDatasetMBean
+  public long getNumBlocksFailedToUncache() {
+    return cacheManager.getNumBlocksFailedToUncache();
+  }
+
+  @Override // FSDatasetMBean
+  public long getNumBlocksCached() {
+    return cacheManager.getNumBlocksCached();
+  }
+
   /**
   /**
    * Find the block's on-disk length
    * Find the block's on-disk length
    */
    */
@@ -571,6 +599,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   private synchronized ReplicaBeingWritten append(String bpid,
   private synchronized ReplicaBeingWritten append(String bpid,
       FinalizedReplica replicaInfo, long newGS, long estimateBlockLen)
       FinalizedReplica replicaInfo, long newGS, long estimateBlockLen)
       throws IOException {
       throws IOException {
+    // If the block is cached, start uncaching it.
+    cacheManager.uncacheBlock(bpid, replicaInfo.getBlockId());
     // unlink the finalized replica
     // unlink the finalized replica
     replicaInfo.unlinkBlock(1);
     replicaInfo.unlinkBlock(1);
     
     
@@ -1050,6 +1080,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     return blockReportsMap;
     return blockReportsMap;
   }
   }
 
 
+  @Override // FsDatasetSpi
+  public List<Long> getCacheReport(String bpid) {
+    return cacheManager.getCachedBlocks(bpid);
+  }
+
   /**
   /**
    * Get the list of finalized blocks from in-memory blockmap for a block pool.
    * Get the list of finalized blocks from in-memory blockmap for a block pool.
    */
    */
@@ -1192,8 +1227,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         }
         }
         volumeMap.remove(bpid, invalidBlks[i]);
         volumeMap.remove(bpid, invalidBlks[i]);
       }
       }
-
-      // Delete the block asynchronously to make sure we can do it fast enough
+      // If the block is cached, start uncaching it.
+      cacheManager.uncacheBlock(bpid, invalidBlks[i].getBlockId());
+      // Delete the block asynchronously to make sure we can do it fast enough.
+      // It's ok to unlink the block file before the uncache operation
+      // finishes.
       asyncDiskService.deleteAsync(v, f,
       asyncDiskService.deleteAsync(v, f,
           FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()),
           FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()),
           new ExtendedBlock(bpid, invalidBlks[i]));
           new ExtendedBlock(bpid, invalidBlks[i]));
@@ -1203,6 +1241,71 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
     }
   }
   }
 
 
+  /**
+   * Asynchronously attempts to cache a single block via {@link FsDatasetCache}.
+   */
+  private void cacheBlock(String bpid, long blockId) {
+    FsVolumeImpl volume;
+    String blockFileName;
+    long length, genstamp;
+    Executor volumeExecutor;
+
+    synchronized (this) {
+      ReplicaInfo info = volumeMap.get(bpid, blockId);
+      boolean success = false;
+      try {
+        if (info == null) {
+          LOG.warn("Failed to cache block with id " + blockId + ", pool " +
+              bpid + ": ReplicaInfo not found.");
+          return;
+        }
+        if (info.getState() != ReplicaState.FINALIZED) {
+          LOG.warn("Failed to cache block with id " + blockId + ", pool " +
+              bpid + ": replica is not finalized; it is in state " +
+              info.getState());
+          return;
+        }
+        try {
+          volume = (FsVolumeImpl)info.getVolume();
+          if (volume == null) {
+            LOG.warn("Failed to cache block with id " + blockId + ", pool " +
+                bpid + ": volume not found.");
+            return;
+          }
+        } catch (ClassCastException e) {
+          LOG.warn("Failed to cache block with id " + blockId +
+              ": volume was not an instance of FsVolumeImpl.");
+          return;
+        }
+        success = true;
+      } finally {
+        if (!success) {
+          cacheManager.numBlocksFailedToCache.incrementAndGet();
+        }
+      }
+      blockFileName = info.getBlockFile().getAbsolutePath();
+      length = info.getVisibleLength();
+      genstamp = info.getGenerationStamp();
+      volumeExecutor = volume.getCacheExecutor();
+    }
+    cacheManager.cacheBlock(blockId, bpid, 
+        blockFileName, length, genstamp, volumeExecutor);
+  }
+
+  @Override // FsDatasetSpi
+  public void cache(String bpid, long[] blockIds) {
+    for (int i=0; i < blockIds.length; i++) {
+      cacheBlock(bpid, blockIds[i]);
+    }
+  }
+
+  @Override // FsDatasetSpi
+  public void uncache(String bpid, long[] blockIds) {
+    for (int i=0; i < blockIds.length; i++) {
+      cacheManager.uncacheBlock(bpid, blockIds[i]);
+    }
+  }
+
   @Override // FsDatasetSpi
   @Override // FsDatasetSpi
   public synchronized boolean contains(final ExtendedBlock block) {
   public synchronized boolean contains(final ExtendedBlock block) {
     final long blockId = block.getLocalBlock().getBlockId();
     final long blockId = block.getLocalBlock().getBlockId();

+ 34 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java

@@ -23,6 +23,11 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -36,6 +41,8 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 /**
 /**
  * The underlying volume used to store replica.
  * The underlying volume used to store replica.
  * 
  * 
@@ -51,6 +58,13 @@ class FsVolumeImpl implements FsVolumeSpi {
   private final File currentDir;    // <StorageDirectory>/current
   private final File currentDir;    // <StorageDirectory>/current
   private final DF usage;           
   private final DF usage;           
   private final long reserved;
   private final long reserved;
+  /**
+   * Per-volume worker pool that processes new blocks to cache.
+   * The maximum number of workers per volume is bounded (configurable via
+   * dfs.datanode.fsdatasetcache.max.threads.per.volume) to limit resource
+   * contention.
+   */
+  private final ThreadPoolExecutor cacheExecutor;
   
   
   FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir,
   FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir,
       Configuration conf, StorageType storageType) throws IOException {
       Configuration conf, StorageType storageType) throws IOException {
@@ -63,6 +77,20 @@ class FsVolumeImpl implements FsVolumeSpi {
     File parent = currentDir.getParentFile();
     File parent = currentDir.getParentFile();
     this.usage = new DF(parent, conf);
     this.usage = new DF(parent, conf);
     this.storageType = storageType;
     this.storageType = storageType;
+    final int maxNumThreads = dataset.datanode.getConf().getInt(
+        DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY,
+        DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT
+        );
+    ThreadFactory workerFactory = new ThreadFactoryBuilder()
+        .setDaemon(true)
+        .setNameFormat("FsVolumeImplWorker-" + parent.toString() + "-%d")
+        .build();
+    cacheExecutor = new ThreadPoolExecutor(
+        1, maxNumThreads,
+        60, TimeUnit.SECONDS,
+        new LinkedBlockingQueue<Runnable>(),
+        workerFactory);
+    cacheExecutor.allowCoreThreadTimeOut(true);
   }
   }
   
   
   File getCurrentDir() {
   File getCurrentDir() {
@@ -170,7 +198,11 @@ class FsVolumeImpl implements FsVolumeSpi {
   File addBlock(String bpid, Block b, File f) throws IOException {
   File addBlock(String bpid, Block b, File f) throws IOException {
     return getBlockPoolSlice(bpid).addBlock(b, f);
     return getBlockPoolSlice(bpid).addBlock(b, f);
   }
   }
-    
+
+  Executor getCacheExecutor() {
+    return cacheExecutor;
+  }
+
   void checkDirs() throws DiskErrorException {
   void checkDirs() throws DiskErrorException {
     // TODO:FEDERATION valid synchronization
     // TODO:FEDERATION valid synchronization
     for(BlockPoolSlice s : bpSlices.values()) {
     for(BlockPoolSlice s : bpSlices.values()) {
@@ -214,6 +246,7 @@ class FsVolumeImpl implements FsVolumeSpi {
   }
   }
 
 
   void shutdown() {
   void shutdown() {
+    cacheExecutor.shutdown();
     Set<Entry<String, BlockPoolSlice>> set = bpSlices.entrySet();
     Set<Entry<String, BlockPoolSlice>> set = bpSlices.entrySet();
     for (Entry<String, BlockPoolSlice> entry : set) {
     for (Entry<String, BlockPoolSlice> entry : set) {
       entry.getValue().shutdown();
       entry.getValue().shutdown();

+ 174 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MappableBlock.java

@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.BufferedInputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileChannel.MapMode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.util.DataChecksum;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * Represents an HDFS block that is mmapped by the DataNode.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class MappableBlock implements Closeable {
+  private MappedByteBuffer mmap;
+  private final long length;
+
+  MappableBlock(MappedByteBuffer mmap, long length) {
+    this.mmap = mmap;
+    this.length = length;
+    assert length > 0;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  /**
+   * Load the block.
+   *
+   * mmap and mlock the block, and then verify its checksum.
+   *
+   * @param length         The current length of the block.
+   * @param blockIn        The block input stream.  Should be positioned at the
+   *                       start.  The caller must close this.
+   * @param metaIn         The meta file input stream.  Should be positioned at
+   *                       the start.  The caller must close this.
+   * @param blockFileName  The block file name, for logging purposes.
+   *
+   * @return               The Mappable block.
+   */
+  public static MappableBlock load(long length,
+      FileInputStream blockIn, FileInputStream metaIn,
+      String blockFileName) throws IOException {
+    MappableBlock mappableBlock = null;
+    MappedByteBuffer mmap = null;
+    try {
+      FileChannel blockChannel = blockIn.getChannel();
+      if (blockChannel == null) {
+        throw new IOException("Block InputStream has no FileChannel.");
+      }
+      mmap = blockChannel.map(MapMode.READ_ONLY, 0, length);
+      NativeIO.POSIX.getCacheManipulator().mlock(blockFileName, mmap, length);
+      verifyChecksum(length, metaIn, blockChannel, blockFileName);
+      mappableBlock = new MappableBlock(mmap, length);
+    } finally {
+      if (mappableBlock == null) {
+        if (mmap != null) {
+          NativeIO.POSIX.munmap(mmap); // unmapping also unlocks
+        }
+      }
+    }
+    return mappableBlock;
+  }
+
+  /**
+   * Verifies the block's checksum. This is an I/O intensive operation.
+   * @return if the block was successfully checksummed.
+   */
+  private static void verifyChecksum(long length,
+      FileInputStream metaIn, FileChannel blockChannel, String blockFileName)
+          throws IOException, ChecksumException {
+    // Verify the checksum from the block's meta file
+    // Get the DataChecksum from the meta file header
+    BlockMetadataHeader header =
+        BlockMetadataHeader.readHeader(new DataInputStream(
+            new BufferedInputStream(metaIn, BlockMetadataHeader
+                .getHeaderSize())));
+    FileChannel metaChannel = metaIn.getChannel();
+    if (metaChannel == null) {
+      throw new IOException("Block InputStream meta file has no FileChannel.");
+    }
+    DataChecksum checksum = header.getChecksum();
+    final int bytesPerChecksum = checksum.getBytesPerChecksum();
+    final int checksumSize = checksum.getChecksumSize();
+    final int numChunks = (8*1024*1024) / bytesPerChecksum;
+    ByteBuffer blockBuf = ByteBuffer.allocate(numChunks*bytesPerChecksum);
+    ByteBuffer checksumBuf = ByteBuffer.allocate(numChunks*checksumSize);
+    // Verify the checksum
+    int bytesVerified = 0;
+    while (bytesVerified < length) {
+      Preconditions.checkState(bytesVerified % bytesPerChecksum == 0,
+          "Unexpected partial chunk before EOF");
+      assert bytesVerified % bytesPerChecksum == 0;
+      int bytesRead = fillBuffer(blockChannel, blockBuf);
+      if (bytesRead == -1) {
+        throw new IOException("checksum verification failed: premature EOF");
+      }
+      blockBuf.flip();
+      // Number of read chunks, including partial chunk at end
+      int chunks = (bytesRead+bytesPerChecksum-1) / bytesPerChecksum;
+      checksumBuf.limit(chunks*checksumSize);
+      fillBuffer(metaChannel, checksumBuf);
+      checksumBuf.flip();
+      checksum.verifyChunkedSums(blockBuf, checksumBuf, blockFileName,
+          bytesVerified);
+      // Success
+      bytesVerified += bytesRead;
+      blockBuf.clear();
+      checksumBuf.clear();
+    }
+  }
+
+  /**
+   * Reads bytes into a buffer until EOF or the buffer's limit is reached
+   */
+  private static int fillBuffer(FileChannel channel, ByteBuffer buf)
+      throws IOException {
+    int bytesRead = channel.read(buf);
+    if (bytesRead < 0) {
+      //EOF
+      return bytesRead;
+    }
+    while (buf.remaining() > 0) {
+      int n = channel.read(buf);
+      if (n < 0) {
+        //EOF
+        return bytesRead;
+      }
+      bytesRead += n;
+    }
+    return bytesRead;
+  }
+
+  @Override
+  public void close() {
+    if (mmap != null) {
+      NativeIO.POSIX.munmap(mmap);
+      mmap = null;
+    }
+  }
+}

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

@@ -57,6 +57,8 @@ public class DataNodeMetrics {
   @Metric MutableCounterLong blocksRemoved;
   @Metric MutableCounterLong blocksRemoved;
   @Metric MutableCounterLong blocksVerified;
   @Metric MutableCounterLong blocksVerified;
   @Metric MutableCounterLong blockVerificationFailures;
   @Metric MutableCounterLong blockVerificationFailures;
+  @Metric MutableCounterLong blocksCached;
+  @Metric MutableCounterLong blocksUncached;
   @Metric MutableCounterLong readsFromLocalClient;
   @Metric MutableCounterLong readsFromLocalClient;
   @Metric MutableCounterLong readsFromRemoteClient;
   @Metric MutableCounterLong readsFromRemoteClient;
   @Metric MutableCounterLong writesFromLocalClient;
   @Metric MutableCounterLong writesFromLocalClient;
@@ -74,6 +76,7 @@ public class DataNodeMetrics {
   @Metric MutableRate replaceBlockOp;
   @Metric MutableRate replaceBlockOp;
   @Metric MutableRate heartbeats;
   @Metric MutableRate heartbeats;
   @Metric MutableRate blockReports;
   @Metric MutableRate blockReports;
+  @Metric MutableRate cacheReports;
   @Metric MutableRate packetAckRoundTripTimeNanos;
   @Metric MutableRate packetAckRoundTripTimeNanos;
   MutableQuantiles[] packetAckRoundTripTimeNanosQuantiles;
   MutableQuantiles[] packetAckRoundTripTimeNanosQuantiles;
   
   
@@ -151,6 +154,10 @@ public class DataNodeMetrics {
     blockReports.add(latency);
     blockReports.add(latency);
   }
   }
 
 
+  public void addCacheReport(long latency) {
+    cacheReports.add(latency);
+  }
+
   public void incrBlocksReplicated(int delta) {
   public void incrBlocksReplicated(int delta) {
     blocksReplicated.incr(delta);
     blocksReplicated.incr(delta);
   }
   }
@@ -175,6 +182,15 @@ public class DataNodeMetrics {
     blocksVerified.incr();
     blocksVerified.incr();
   }
   }
 
 
+
+  public void incrBlocksCached(int delta) {
+    blocksCached.incr(delta);
+  }
+
+  public void incrBlocksUncached(int delta) {
+    blocksUncached.incr(delta);
+  }
+
   public void addReadBlockOp(long latency) {
   public void addReadBlockOp(long latency) {
     readBlockOp.add(latency);
     readBlockOp.add(latency);
   }
   }

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

@@ -76,4 +76,29 @@ public interface FSDatasetMBean {
    * @return The number of failed volumes in the datanode.
    * @return The number of failed volumes in the datanode.
    */
    */
   public int getNumFailedVolumes();
   public int getNumFailedVolumes();
+
+  /**
+   * Returns the amount of cache used by the datanode (in bytes).
+   */
+  public long getCacheUsed();
+
+  /**
+   * Returns the total cache capacity of the datanode (in bytes).
+   */
+  public long getCacheCapacity();
+
+  /**
+   * Returns the number of blocks cached.
+   */
+  public long getNumBlocksCached();
+
+  /**
+   * Returns the number of blocks that the datanode was unable to cache
+   */
+  public long getNumBlocksFailedToCache();
+
+  /**
+   * Returns the number of blocks that the datanode was unable to uncache
+   */
+  public long getNumBlocksFailedToUncache();
 }
 }

+ 1073 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -0,0 +1,1073 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.CacheDirective;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
+import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.util.GSet;
+import org.apache.hadoop.util.LightWeightGSet;
+import org.apache.hadoop.util.Time;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * The Cache Manager handles caching on DataNodes.
+ *
+ * This class is instantiated by the FSNamesystem.
+ * It maintains the mapping of cached blocks to datanodes via processing
+ * datanode cache reports. Based on these reports and addition and removal of
+ * caching directives, we will schedule caching and uncaching work.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+public final class CacheManager {
+  public static final Log LOG = LogFactory.getLog(CacheManager.class);
+
+  private static final float MIN_CACHED_BLOCKS_PERCENT = 0.001f;
+
+  // TODO: add pending / underCached / schedule cached blocks stats.
+
+  /**
+   * The FSNamesystem that contains this CacheManager.
+   */
+  private final FSNamesystem namesystem;
+
+  /**
+   * The BlockManager associated with the FSN that owns this CacheManager.
+   */
+  private final BlockManager blockManager;
+
+  /**
+   * Cache directives, sorted by ID.
+   *
+   * listCacheDirectives relies on the ordering of elements in this map
+   * to track what has already been listed by the client.
+   */
+  private final TreeMap<Long, CacheDirective> directivesById =
+      new TreeMap<Long, CacheDirective>();
+
+  /**
+   * The directive ID to use for a new directive.  IDs always increase, and are
+   * never reused.
+   */
+  private long nextDirectiveId;
+
+  /**
+   * Cache directives, sorted by path
+   */
+  private final TreeMap<String, List<CacheDirective>> directivesByPath =
+      new TreeMap<String, List<CacheDirective>>();
+
+  /**
+   * Cache pools, sorted by name.
+   */
+  private final TreeMap<String, CachePool> cachePools =
+      new TreeMap<String, CachePool>();
+
+  /**
+   * Maximum number of cache pools to list in one operation.
+   */
+  private final int maxListCachePoolsResponses;
+
+  /**
+   * Maximum number of cache pool directives to list in one operation.
+   */
+  private final int maxListCacheDirectivesNumResponses;
+
+  /**
+   * Interval between scans in milliseconds.
+   */
+  private final long scanIntervalMs;
+
+  /**
+   * All cached blocks.
+   */
+  private final GSet<CachedBlock, CachedBlock> cachedBlocks;
+
+  /**
+   * Lock which protects the CacheReplicationMonitor.
+   */
+  private final ReentrantLock crmLock = new ReentrantLock();
+
+  /**
+   * The CacheReplicationMonitor.
+   */
+  private CacheReplicationMonitor monitor;
+
+  CacheManager(FSNamesystem namesystem, Configuration conf,
+      BlockManager blockManager) {
+    this.namesystem = namesystem;
+    this.blockManager = blockManager;
+    this.nextDirectiveId = 1;
+    this.maxListCachePoolsResponses = conf.getInt(
+        DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES,
+        DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT);
+    this.maxListCacheDirectivesNumResponses = conf.getInt(
+        DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES,
+        DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES_DEFAULT);
+    scanIntervalMs = conf.getLong(
+        DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS,
+        DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT);
+    float cachedBlocksPercent = conf.getFloat(
+          DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT,
+          DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT_DEFAULT);
+    if (cachedBlocksPercent < MIN_CACHED_BLOCKS_PERCENT) {
+      LOG.info("Using minimum value " + MIN_CACHED_BLOCKS_PERCENT +
+        " for " + DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT);
+      cachedBlocksPercent = MIN_CACHED_BLOCKS_PERCENT;
+    }
+    this.cachedBlocks = new LightWeightGSet<CachedBlock, CachedBlock>(
+          LightWeightGSet.computeCapacity(cachedBlocksPercent,
+              "cachedBlocks"));
+
+  }
+
+  public void startMonitorThread() {
+    crmLock.lock();
+    try {
+      if (this.monitor == null) {
+        this.monitor = new CacheReplicationMonitor(namesystem, this,
+            scanIntervalMs, crmLock);
+        this.monitor.start();
+      }
+    } finally {
+      crmLock.unlock();
+    }
+  }
+
+  public void stopMonitorThread() {
+    crmLock.lock();
+    try {
+      if (this.monitor != null) {
+        CacheReplicationMonitor prevMonitor = this.monitor;
+        this.monitor = null;
+        IOUtils.closeQuietly(prevMonitor);
+      }
+    } finally {
+      crmLock.unlock();
+    }
+  }
+
+  public void clearDirectiveStats() {
+    assert namesystem.hasWriteLock();
+    for (CacheDirective directive : directivesById.values()) {
+      directive.resetStatistics();
+    }
+  }
+
+  /**
+   * @return Unmodifiable view of the collection of CachePools.
+   */
+  public Collection<CachePool> getCachePools() {
+    assert namesystem.hasReadLock();
+    return Collections.unmodifiableCollection(cachePools.values());
+  }
+
+  /**
+   * @return Unmodifiable view of the collection of CacheDirectives.
+   */
+  public Collection<CacheDirective> getCacheDirectives() {
+    assert namesystem.hasReadLock();
+    return Collections.unmodifiableCollection(directivesById.values());
+  }
+  
+  @VisibleForTesting
+  public GSet<CachedBlock, CachedBlock> getCachedBlocks() {
+    assert namesystem.hasReadLock();
+    return cachedBlocks;
+  }
+
+  private long getNextDirectiveId() throws IOException {
+    assert namesystem.hasWriteLock();
+    if (nextDirectiveId >= Long.MAX_VALUE - 1) {
+      throw new IOException("No more available IDs.");
+    }
+    return nextDirectiveId++;
+  }
+
+  // Helper getter / validation methods
+
+  private static void checkWritePermission(FSPermissionChecker pc,
+      CachePool pool) throws AccessControlException {
+    if ((pc != null)) {
+      pc.checkPermission(pool, FsAction.WRITE);
+    }
+  }
+
+  private static String validatePoolName(CacheDirectiveInfo directive)
+      throws InvalidRequestException {
+    String pool = directive.getPool();
+    if (pool == null) {
+      throw new InvalidRequestException("No pool specified.");
+    }
+    if (pool.isEmpty()) {
+      throw new InvalidRequestException("Invalid empty pool name.");
+    }
+    return pool;
+  }
+
+  private static String validatePath(CacheDirectiveInfo directive)
+      throws InvalidRequestException {
+    if (directive.getPath() == null) {
+      throw new InvalidRequestException("No path specified.");
+    }
+    String path = directive.getPath().toUri().getPath();
+    if (!DFSUtil.isValidName(path)) {
+      throw new InvalidRequestException("Invalid path '" + path + "'.");
+    }
+    return path;
+  }
+
+  private static short validateReplication(CacheDirectiveInfo directive,
+      short defaultValue) throws InvalidRequestException {
+    short repl = (directive.getReplication() != null)
+        ? directive.getReplication() : defaultValue;
+    if (repl <= 0) {
+      throw new InvalidRequestException("Invalid replication factor " + repl
+          + " <= 0");
+    }
+    return repl;
+  }
+
+  /**
+   * Calculates the absolute expiry time of the directive from the
+   * {@link CacheDirectiveInfo.Expiration}. This converts a relative Expiration
+   * into an absolute time based on the local clock.
+   * 
+   * @param info to validate.
+   * @param maxRelativeExpiryTime of the info's pool.
+   * @return the expiration time, or the pool's max absolute expiration if the
+   *         info's expiration was not set.
+   * @throws InvalidRequestException if the info's Expiration is invalid.
+   */
+  private static long validateExpiryTime(CacheDirectiveInfo info,
+      long maxRelativeExpiryTime) throws InvalidRequestException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Validating directive " + info
+          + " pool maxRelativeExpiryTime " + maxRelativeExpiryTime);
+    }
+    final long now = new Date().getTime();
+    final long maxAbsoluteExpiryTime = now + maxRelativeExpiryTime;
+    if (info == null || info.getExpiration() == null) {
+      return maxAbsoluteExpiryTime;
+    }
+    Expiration expiry = info.getExpiration();
+    if (expiry.getMillis() < 0l) {
+      throw new InvalidRequestException("Cannot set a negative expiration: "
+          + expiry.getMillis());
+    }
+    long relExpiryTime, absExpiryTime;
+    if (expiry.isRelative()) {
+      relExpiryTime = expiry.getMillis();
+      absExpiryTime = now + relExpiryTime;
+    } else {
+      absExpiryTime = expiry.getMillis();
+      relExpiryTime = absExpiryTime - now;
+    }
+    // Need to cap the expiry so we don't overflow a long when doing math
+    if (relExpiryTime > Expiration.MAX_RELATIVE_EXPIRY_MS) {
+      throw new InvalidRequestException("Expiration "
+          + expiry.toString() + " is too far in the future!");
+    }
+    // Fail if the requested expiry is greater than the max
+    if (relExpiryTime > maxRelativeExpiryTime) {
+      throw new InvalidRequestException("Expiration " + expiry.toString()
+          + " exceeds the max relative expiration time of "
+          + maxRelativeExpiryTime + " ms.");
+    }
+    return absExpiryTime;
+  }
+
+  /**
+   * 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.getLimit() == CachePoolInfo.LIMIT_UNLIMITED) {
+      return;
+    }
+    if (pool.getBytesNeeded() + (stats.getBytesNeeded() * replication) > pool
+        .getLimit()) {
+      throw new InvalidRequestException("Caching path " + path + " of size "
+          + stats.getBytesNeeded() / replication + " bytes at replication "
+          + replication + " would exceed pool " + pool.getPoolName()
+          + "'s remaining capacity of "
+          + (pool.getLimit() - pool.getBytesNeeded()) + " bytes.");
+    }
+  }
+
+  /**
+   * Computes the needed number of bytes and files for a path.
+   * @return CacheDirectiveStats describing the needed stats for this path
+   */
+  private CacheDirectiveStats computeNeeded(String path, short replication) {
+    FSDirectory fsDir = namesystem.getFSDirectory();
+    INode node;
+    long requestedBytes = 0;
+    long requestedFiles = 0;
+    CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
+    try {
+      node = fsDir.getINode(path);
+    } catch (UnresolvedLinkException e) {
+      // We don't cache through symlinks
+      return builder.build();
+    }
+    if (node == null) {
+      return builder.build();
+    }
+    if (node.isFile()) {
+      requestedFiles = 1;
+      INodeFile file = node.asFile();
+      requestedBytes = file.computeFileSize();
+    } else if (node.isDirectory()) {
+      INodeDirectory dir = node.asDirectory();
+      ReadOnlyList<INode> children = dir.getChildrenList(null);
+      requestedFiles = children.size();
+      for (INode child : children) {
+        if (child.isFile()) {
+          requestedBytes += child.asFile().computeFileSize();
+        }
+      }
+    }
+    return new CacheDirectiveStats.Builder()
+        .setBytesNeeded(requestedBytes)
+        .setFilesCached(requestedFiles)
+        .build();
+  }
+
+  /**
+   * Get a CacheDirective by ID, validating the ID and that the directive
+   * exists.
+   */
+  private CacheDirective getById(long id) throws InvalidRequestException {
+    // Check for invalid IDs.
+    if (id <= 0) {
+      throw new InvalidRequestException("Invalid negative ID.");
+    }
+    // Find the directive.
+    CacheDirective directive = directivesById.get(id);
+    if (directive == null) {
+      throw new InvalidRequestException("No directive with ID " + id
+          + " found.");
+    }
+    return directive;
+  }
+
+  /**
+   * Get a CachePool by name, validating that it exists.
+   */
+  private CachePool getCachePool(String poolName)
+      throws InvalidRequestException {
+    CachePool pool = cachePools.get(poolName);
+    if (pool == null) {
+      throw new InvalidRequestException("Unknown pool " + poolName);
+    }
+    return pool;
+  }
+
+  // RPC handlers
+
+  private void addInternal(CacheDirective directive, CachePool pool) {
+    boolean addedDirective = pool.getDirectiveList().add(directive);
+    assert addedDirective;
+    directivesById.put(directive.getId(), directive);
+    String path = directive.getPath();
+    List<CacheDirective> directives = directivesByPath.get(path);
+    if (directives == null) {
+      directives = new ArrayList<CacheDirective>(1);
+      directivesByPath.put(path, directives);
+    }
+    directives.add(directive);
+    // Fix up pool stats
+    CacheDirectiveStats stats =
+        computeNeeded(directive.getPath(), directive.getReplication());
+    directive.addBytesNeeded(stats.getBytesNeeded());
+    directive.addFilesNeeded(directive.getFilesNeeded());
+
+    setNeedsRescan();
+  }
+
+  /**
+   * Adds a directive, skipping most error checking. This should only be called
+   * internally in special scenarios like edit log replay.
+   */
+  CacheDirectiveInfo addDirectiveFromEditLog(CacheDirectiveInfo directive)
+      throws InvalidRequestException {
+    long id = directive.getId();
+    CacheDirective entry = new CacheDirective(directive);
+    CachePool pool = cachePools.get(directive.getPool());
+    addInternal(entry, pool);
+    if (nextDirectiveId <= id) {
+      nextDirectiveId = id + 1;
+    }
+    return entry.toInfo();
+  }
+
+  public CacheDirectiveInfo addDirective(
+      CacheDirectiveInfo info, FSPermissionChecker pc, EnumSet<CacheFlag> flags)
+      throws IOException {
+    assert namesystem.hasWriteLock();
+    CacheDirective directive;
+    try {
+      CachePool pool = getCachePool(validatePoolName(info));
+      checkWritePermission(pc, pool);
+      String path = validatePath(info);
+      short replication = validateReplication(info, (short)1);
+      long expiryTime = validateExpiryTime(info, pool.getMaxRelativeExpiryMs());
+      // Do quota validation if required
+      if (!flags.contains(CacheFlag.FORCE)) {
+        checkLimit(pool, path, replication);
+      }
+      // All validation passed
+      // Add a new entry with the next available ID.
+      long id = getNextDirectiveId();
+      directive = new CacheDirective(id, path, replication, expiryTime);
+      addInternal(directive, pool);
+    } catch (IOException e) {
+      LOG.warn("addDirective of " + info + " failed: ", e);
+      throw e;
+    }
+    LOG.info("addDirective of " + info + " successful.");
+    return directive.toInfo();
+  }
+
+  /**
+   * Factory method that makes a new CacheDirectiveInfo by applying fields in a
+   * CacheDirectiveInfo to an existing CacheDirective.
+   * 
+   * @param info with some or all fields set.
+   * @param defaults directive providing default values for unset fields in
+   *          info.
+   * 
+   * @return new CacheDirectiveInfo of the info applied to the defaults.
+   */
+  private static CacheDirectiveInfo createFromInfoAndDefaults(
+      CacheDirectiveInfo info, CacheDirective defaults) {
+    // Initialize the builder with the default values
+    CacheDirectiveInfo.Builder builder =
+        new CacheDirectiveInfo.Builder(defaults.toInfo());
+    // Replace default with new value if present
+    if (info.getPath() != null) {
+      builder.setPath(info.getPath());
+    }
+    if (info.getReplication() != null) {
+      builder.setReplication(info.getReplication());
+    }
+    if (info.getPool() != null) {
+      builder.setPool(info.getPool());
+    }
+    if (info.getExpiration() != null) {
+      builder.setExpiration(info.getExpiration());
+    }
+    return builder.build();
+  }
+
+  /**
+   * Modifies a directive, skipping most error checking. This is for careful
+   * internal use only. modifyDirective can be non-deterministic since its error
+   * checking depends on current system time, which poses a problem for edit log
+   * replay.
+   */
+  void modifyDirectiveFromEditLog(CacheDirectiveInfo info)
+      throws InvalidRequestException {
+    // Check for invalid IDs.
+    Long id = info.getId();
+    if (id == null) {
+      throw new InvalidRequestException("Must supply an ID.");
+    }
+    CacheDirective prevEntry = getById(id);
+    CacheDirectiveInfo newInfo = createFromInfoAndDefaults(info, prevEntry);
+    removeInternal(prevEntry);
+    addInternal(new CacheDirective(newInfo), getCachePool(newInfo.getPool()));
+  }
+
+  public void modifyDirective(CacheDirectiveInfo info,
+      FSPermissionChecker pc, EnumSet<CacheFlag> flags) throws IOException {
+    assert namesystem.hasWriteLock();
+    String idString =
+        (info.getId() == null) ?
+            "(null)" : info.getId().toString();
+    try {
+      // Check for invalid IDs.
+      Long id = info.getId();
+      if (id == null) {
+        throw new InvalidRequestException("Must supply an ID.");
+      }
+      CacheDirective prevEntry = getById(id);
+      checkWritePermission(pc, prevEntry.getPool());
+
+      // Fill in defaults
+      CacheDirectiveInfo infoWithDefaults =
+          createFromInfoAndDefaults(info, prevEntry);
+      CacheDirectiveInfo.Builder builder =
+          new CacheDirectiveInfo.Builder(infoWithDefaults);
+
+      // Do validation
+      validatePath(infoWithDefaults);
+      validateReplication(infoWithDefaults, (short)-1);
+      // Need to test the pool being set here to avoid rejecting a modify for a
+      // directive that's already been forced into a pool
+      CachePool srcPool = prevEntry.getPool();
+      CachePool destPool = getCachePool(validatePoolName(infoWithDefaults));
+      if (!srcPool.getPoolName().equals(destPool.getPoolName())) {
+        checkWritePermission(pc, destPool);
+        if (!flags.contains(CacheFlag.FORCE)) {
+          checkLimit(destPool, infoWithDefaults.getPath().toUri().getPath(),
+              infoWithDefaults.getReplication());
+        }
+      }
+      // Verify the expiration against the destination pool
+      validateExpiryTime(infoWithDefaults, destPool.getMaxRelativeExpiryMs());
+
+      // Indicate changes to the CRM
+      setNeedsRescan();
+
+      // Validation passed
+      removeInternal(prevEntry);
+      addInternal(new CacheDirective(builder.build()), destPool);
+    } catch (IOException e) {
+      LOG.warn("modifyDirective of " + idString + " failed: ", e);
+      throw e;
+    }
+    LOG.info("modifyDirective of " + idString + " successfully applied " +
+        info+ ".");
+  }
+
+  private void removeInternal(CacheDirective directive)
+      throws InvalidRequestException {
+    assert namesystem.hasWriteLock();
+    // Remove the corresponding entry in directivesByPath.
+    String path = directive.getPath();
+    List<CacheDirective> directives = directivesByPath.get(path);
+    if (directives == null || !directives.remove(directive)) {
+      throw new InvalidRequestException("Failed to locate entry " +
+          directive.getId() + " by path " + directive.getPath());
+    }
+    if (directives.size() == 0) {
+      directivesByPath.remove(path);
+    }
+    // Fix up the stats from removing the pool
+    final CachePool pool = directive.getPool();
+    directive.addBytesNeeded(-directive.getBytesNeeded());
+    directive.addFilesNeeded(-directive.getFilesNeeded());
+
+    directivesById.remove(directive.getId());
+    pool.getDirectiveList().remove(directive);
+    assert directive.getPool() == null;
+
+    setNeedsRescan();
+  }
+
+  public void removeDirective(long id, FSPermissionChecker pc)
+      throws IOException {
+    assert namesystem.hasWriteLock();
+    try {
+      CacheDirective directive = getById(id);
+      checkWritePermission(pc, directive.getPool());
+      removeInternal(directive);
+    } catch (IOException e) {
+      LOG.warn("removeDirective of " + id + " failed: ", e);
+      throw e;
+    }
+    LOG.info("removeDirective of " + id + " successful.");
+  }
+
+  public BatchedListEntries<CacheDirectiveEntry> 
+        listCacheDirectives(long prevId,
+            CacheDirectiveInfo filter,
+            FSPermissionChecker pc) throws IOException {
+    assert namesystem.hasReadLock();
+    final int NUM_PRE_ALLOCATED_ENTRIES = 16;
+    String filterPath = null;
+    if (filter.getId() != null) {
+      throw new IOException("Filtering by ID is unsupported.");
+    }
+    if (filter.getPath() != null) {
+      filterPath = validatePath(filter);
+    }
+    if (filter.getReplication() != null) {
+      throw new IOException("Filtering by replication is unsupported.");
+    }
+    ArrayList<CacheDirectiveEntry> replies =
+        new ArrayList<CacheDirectiveEntry>(NUM_PRE_ALLOCATED_ENTRIES);
+    int numReplies = 0;
+    SortedMap<Long, CacheDirective> tailMap =
+      directivesById.tailMap(prevId + 1);
+    for (Entry<Long, CacheDirective> cur : tailMap.entrySet()) {
+      if (numReplies >= maxListCacheDirectivesNumResponses) {
+        return new BatchedListEntries<CacheDirectiveEntry>(replies, true);
+      }
+      CacheDirective curDirective = cur.getValue();
+      CacheDirectiveInfo info = cur.getValue().toInfo();
+      if (filter.getPool() != null && 
+          !info.getPool().equals(filter.getPool())) {
+        continue;
+      }
+      if (filterPath != null &&
+          !info.getPath().toUri().getPath().equals(filterPath)) {
+        continue;
+      }
+      boolean hasPermission = true;
+      if (pc != null) {
+        try {
+          pc.checkPermission(curDirective.getPool(), FsAction.READ);
+        } catch (AccessControlException e) {
+          hasPermission = false;
+        }
+      }
+      if (hasPermission) {
+        replies.add(new CacheDirectiveEntry(info, cur.getValue().toStats()));
+        numReplies++;
+      }
+    }
+    return new BatchedListEntries<CacheDirectiveEntry>(replies, false);
+  }
+
+  /**
+   * Create a cache pool.
+   * 
+   * Only the superuser should be able to call this function.
+   *
+   * @param info    The info for the cache pool to create.
+   * @return        Information about the cache pool we created.
+   */
+  public CachePoolInfo addCachePool(CachePoolInfo info)
+      throws IOException {
+    assert namesystem.hasWriteLock();
+    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;
+    }
+    LOG.info("addCachePool of " + info + " successful.");
+    return pool.getInfo(true);
+  }
+
+  /**
+   * Modify a cache pool.
+   * 
+   * Only the superuser should be able to call this function.
+   *
+   * @param info
+   *          The info for the cache pool to modify.
+   */
+  public void modifyCachePool(CachePoolInfo info)
+      throws IOException {
+    assert namesystem.hasWriteLock();
+    StringBuilder bld = new StringBuilder();
+    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
+        setNeedsRescan();
+      }
+      if (info.getMaxRelativeExpiryMs() != null) {
+        final Long maxRelativeExpiry = info.getMaxRelativeExpiryMs();
+        pool.setMaxRelativeExpiryMs(maxRelativeExpiry);
+        bld.append(prefix).append("set maxRelativeExpiry to "
+            + maxRelativeExpiry);
+        prefix = "; ";
+      }
+      if (prefix.isEmpty()) {
+        bld.append("no changes.");
+      }
+    } catch (IOException e) {
+      LOG.info("modifyCachePool of " + info + " failed: ", e);
+      throw e;
+    }
+    LOG.info("modifyCachePool of " + info.getPoolName() + " successful; "
+        + bld.toString());
+  }
+
+  /**
+   * Remove a cache pool.
+   * 
+   * Only the superuser should be able to call this function.
+   *
+   * @param poolName
+   *          The name for the cache pool to remove.
+   */
+  public void removeCachePool(String poolName)
+      throws IOException {
+    assert namesystem.hasWriteLock();
+    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();
+      }
+      setNeedsRescan();
+    } catch (IOException e) {
+      LOG.info("removeCachePool of " + poolName + " failed: ", e);
+      throw e;
+    }
+    LOG.info("removeCachePool of " + poolName + " successful.");
+  }
+
+  public BatchedListEntries<CachePoolEntry>
+      listCachePools(FSPermissionChecker pc, String prevKey) {
+    assert namesystem.hasReadLock();
+    final int NUM_PRE_ALLOCATED_ENTRIES = 16;
+    ArrayList<CachePoolEntry> results = 
+        new ArrayList<CachePoolEntry>(NUM_PRE_ALLOCATED_ENTRIES);
+    SortedMap<String, CachePool> tailMap = cachePools.tailMap(prevKey, false);
+    int numListed = 0;
+    for (Entry<String, CachePool> cur : tailMap.entrySet()) {
+      if (numListed++ >= maxListCachePoolsResponses) {
+        return new BatchedListEntries<CachePoolEntry>(results, true);
+      }
+      results.add(cur.getValue().getEntry(pc));
+    }
+    return new BatchedListEntries<CachePoolEntry>(results, false);
+  }
+
+  public void setCachedLocations(LocatedBlock block) {
+    CachedBlock cachedBlock =
+        new CachedBlock(block.getBlock().getBlockId(),
+            (short)0, false);
+    cachedBlock = cachedBlocks.get(cachedBlock);
+    if (cachedBlock == null) {
+      return;
+    }
+    List<DatanodeDescriptor> datanodes = cachedBlock.getDatanodes(Type.CACHED);
+    for (DatanodeDescriptor datanode : datanodes) {
+      block.addCachedLoc(datanode);
+    }
+  }
+
+  public final void processCacheReport(final DatanodeID datanodeID,
+      final List<Long> blockIds) throws IOException {
+    namesystem.writeLock();
+    final long startTime = Time.monotonicNow();
+    final long endTime;
+    try {
+      final DatanodeDescriptor datanode = 
+          blockManager.getDatanodeManager().getDatanode(datanodeID);
+      if (datanode == null || !datanode.isAlive) {
+        throw new IOException(
+            "processCacheReport from dead or unregistered datanode: " +
+            datanode);
+      }
+      processCacheReportImpl(datanode, blockIds);
+    } finally {
+      endTime = Time.monotonicNow();
+      namesystem.writeUnlock();
+    }
+
+    // Log the block report processing stats from Namenode perspective
+    final NameNodeMetrics metrics = NameNode.getNameNodeMetrics();
+    if (metrics != null) {
+      metrics.addCacheBlockReport((int) (endTime - startTime));
+    }
+    LOG.info("Processed cache report from "
+        + datanodeID + ", blocks: " + blockIds.size()
+        + ", processing time: " + (endTime - startTime) + " msecs");
+  }
+
+  private void processCacheReportImpl(final DatanodeDescriptor datanode,
+      final List<Long> blockIds) {
+    CachedBlocksList cached = datanode.getCached();
+    cached.clear();
+    CachedBlocksList cachedList = datanode.getCached();
+    CachedBlocksList pendingCachedList = datanode.getPendingCached();
+    for (Iterator<Long> iter = blockIds.iterator(); iter.hasNext(); ) {
+      long blockId = iter.next();
+      CachedBlock cachedBlock =
+          new CachedBlock(blockId, (short)0, false);
+      CachedBlock prevCachedBlock = cachedBlocks.get(cachedBlock);
+      // Add the block ID from the cache report to the cachedBlocks map
+      // if it's not already there.
+      if (prevCachedBlock != null) {
+        cachedBlock = prevCachedBlock;
+      } else {
+        cachedBlocks.put(cachedBlock);
+      }
+      // Add the block to the datanode's implicit cached block list
+      // if it's not already there.  Similarly, remove it from the pending
+      // cached block list if it exists there.
+      if (!cachedBlock.isPresent(cachedList)) {
+        cachedList.add(cachedBlock);
+      }
+      if (cachedBlock.isPresent(pendingCachedList)) {
+        pendingCachedList.remove(cachedBlock);
+      }
+    }
+  }
+
+  /**
+   * Saves the current state of the CacheManager to the DataOutput. Used
+   * to persist CacheManager state in the FSImage.
+   * @param out DataOutput to persist state
+   * @param sdPath path of the storage directory
+   * @throws IOException
+   */
+  public void saveState(DataOutputStream out, String sdPath)
+      throws IOException {
+    out.writeLong(nextDirectiveId);
+    savePools(out, sdPath);
+    saveDirectives(out, sdPath);
+  }
+
+  /**
+   * Reloads CacheManager state from the passed DataInput. Used during namenode
+   * startup to restore CacheManager state from an FSImage.
+   * @param in DataInput from which to restore state
+   * @throws IOException
+   */
+  public void loadState(DataInput in) throws IOException {
+    nextDirectiveId = in.readLong();
+    // pools need to be loaded first since directives point to their parent pool
+    loadPools(in);
+    loadDirectives(in);
+  }
+
+  /**
+   * Save cache pools to fsimage
+   */
+  private void savePools(DataOutputStream out,
+      String sdPath) throws IOException {
+    StartupProgress prog = NameNode.getStartupProgress();
+    Step step = new Step(StepType.CACHE_POOLS, sdPath);
+    prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+    prog.setTotal(Phase.SAVING_CHECKPOINT, step, cachePools.size());
+    Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+    out.writeInt(cachePools.size());
+    for (CachePool pool: cachePools.values()) {
+      FSImageSerialization.writeCachePoolInfo(out, pool.getInfo(true));
+      counter.increment();
+    }
+    prog.endStep(Phase.SAVING_CHECKPOINT, step);
+  }
+
+  /*
+   * Save cache entries to fsimage
+   */
+  private void saveDirectives(DataOutputStream out, String sdPath)
+      throws IOException {
+    StartupProgress prog = NameNode.getStartupProgress();
+    Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
+    prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+    prog.setTotal(Phase.SAVING_CHECKPOINT, step, directivesById.size());
+    Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+    out.writeInt(directivesById.size());
+    for (CacheDirective directive : directivesById.values()) {
+      FSImageSerialization.writeCacheDirectiveInfo(out, directive.toInfo());
+      counter.increment();
+    }
+    prog.endStep(Phase.SAVING_CHECKPOINT, step);
+  }
+
+  /**
+   * Load cache pools from fsimage
+   */
+  private void loadPools(DataInput in)
+      throws IOException {
+    StartupProgress prog = NameNode.getStartupProgress();
+    Step step = new Step(StepType.CACHE_POOLS);
+    prog.beginStep(Phase.LOADING_FSIMAGE, step);
+    int numberOfPools = in.readInt();
+    prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfPools);
+    Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
+    for (int i = 0; i < numberOfPools; i++) {
+      addCachePool(FSImageSerialization.readCachePoolInfo(in));
+      counter.increment();
+    }
+    prog.endStep(Phase.LOADING_FSIMAGE, step);
+  }
+
+  /**
+   * Load cache directives from the fsimage
+   */
+  private void loadDirectives(DataInput in) throws IOException {
+    StartupProgress prog = NameNode.getStartupProgress();
+    Step step = new Step(StepType.CACHE_ENTRIES);
+    prog.beginStep(Phase.LOADING_FSIMAGE, step);
+    int numDirectives = in.readInt();
+    prog.setTotal(Phase.LOADING_FSIMAGE, step, numDirectives);
+    Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
+    for (int i = 0; i < numDirectives; i++) {
+      CacheDirectiveInfo info = FSImageSerialization.readCacheDirectiveInfo(in);
+      // Get pool reference by looking it up in the map
+      final String poolName = info.getPool();
+      CachePool pool = cachePools.get(poolName);
+      if (pool == null) {
+        throw new IOException("Directive refers to pool " + poolName +
+            ", which does not exist.");
+      }
+      CacheDirective directive =
+          new CacheDirective(info.getId(), info.getPath().toUri().getPath(),
+              info.getReplication(), info.getExpiration().getAbsoluteMillis());
+      boolean addedDirective = pool.getDirectiveList().add(directive);
+      assert addedDirective;
+      if (directivesById.put(directive.getId(), directive) != null) {
+        throw new IOException("A directive with ID " + directive.getId() +
+            " already exists");
+      }
+      List<CacheDirective> directives =
+          directivesByPath.get(directive.getPath());
+      if (directives == null) {
+        directives = new LinkedList<CacheDirective>();
+        directivesByPath.put(directive.getPath(), directives);
+      }
+      directives.add(directive);
+      counter.increment();
+    }
+    prog.endStep(Phase.LOADING_FSIMAGE, step);
+  }
+
+  public void waitForRescanIfNeeded() {
+    crmLock.lock();
+    try {
+      if (monitor != null) {
+        monitor.waitForRescanIfNeeded();
+      }
+    } finally {
+      crmLock.unlock();
+    }
+  }
+
+  private void setNeedsRescan() {
+    crmLock.lock();
+    try {
+      if (monitor != null) {
+        monitor.setNeedsRescan();
+      }
+    } finally {
+      crmLock.unlock();
+    }
+  }
+
+  @VisibleForTesting
+  public Thread getCacheReplicationMonitor() {
+    crmLock.lock();
+    try {
+      return monitor;
+    } finally {
+      crmLock.unlock();
+    }
+  }
+}

+ 328 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java

@@ -0,0 +1,328 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import javax.annotation.Nonnull;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.CacheDirective;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolStats;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.IntrusiveCollection;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A CachePool describes a set of cache resources being managed by the NameNode.
+ * User caching requests are billed to the cache pool specified in the request.
+ *
+ * This is an internal class, only used on the NameNode.  For identifying or
+ * describing a cache pool to clients, please use CachePoolInfo.
+ * 
+ * CachePools must be accessed under the FSNamesystem lock.
+ */
+@InterfaceAudience.Private
+public final class CachePool {
+  public static final Log LOG = LogFactory.getLog(CachePool.class);
+
+  @Nonnull
+  private final String poolName;
+
+  @Nonnull
+  private String ownerName;
+
+  @Nonnull
+  private String groupName;
+  
+  /**
+   * Cache pool permissions.
+   * 
+   * READ permission means that you can list the cache directives in this pool.
+   * WRITE permission means that you can add, remove, or modify cache directives
+   *       in this pool.
+   * EXECUTE permission is unused.
+   */
+  @Nonnull
+  private FsPermission mode;
+
+  /**
+   * Maximum number of bytes that can be cached in this pool.
+   */
+  private long limit;
+
+  /**
+   * Maximum duration that a CacheDirective in this pool remains valid,
+   * in milliseconds.
+   */
+  private long maxRelativeExpiryMs;
+
+  private long bytesNeeded;
+  private long bytesCached;
+  private long filesNeeded;
+  private long filesCached;
+
+  public final static class DirectiveList
+      extends IntrusiveCollection<CacheDirective> {
+    private CachePool cachePool;
+
+    private DirectiveList(CachePool cachePool) {
+      this.cachePool = cachePool;
+    }
+
+    public CachePool getCachePool() {
+      return cachePool;
+    }
+  }
+
+  @Nonnull
+  private final DirectiveList directiveList = new DirectiveList(this);
+
+  /**
+   * Create a new cache pool based on a CachePoolInfo object and the defaults.
+   * We will fill in information that was not supplied according to the
+   * defaults.
+   */
+  static CachePool createFromInfoAndDefaults(CachePoolInfo info)
+      throws IOException {
+    UserGroupInformation ugi = null;
+    String ownerName = info.getOwnerName();
+    if (ownerName == null) {
+      if (ugi == null) {
+        ugi = NameNode.getRemoteUser();
+      }
+      ownerName = ugi.getShortUserName();
+    }
+    String groupName = info.getGroupName();
+    if (groupName == null) {
+      if (ugi == null) {
+        ugi = NameNode.getRemoteUser();
+      }
+      groupName = ugi.getPrimaryGroupName();
+    }
+    FsPermission mode = (info.getMode() == null) ? 
+        FsPermission.getCachePoolDefault() : info.getMode();
+    long limit = info.getLimit() == null ?
+        CachePoolInfo.DEFAULT_LIMIT : info.getLimit();
+    long maxRelativeExpiry = info.getMaxRelativeExpiryMs() == null ?
+        CachePoolInfo.DEFAULT_MAX_RELATIVE_EXPIRY :
+        info.getMaxRelativeExpiryMs();
+    return new CachePool(info.getPoolName(),
+        ownerName, groupName, mode, limit, maxRelativeExpiry);
+  }
+
+  /**
+   * Create a new cache pool based on a CachePoolInfo object.
+   * No fields in the CachePoolInfo can be blank.
+   */
+  static CachePool createFromInfo(CachePoolInfo info) {
+    return new CachePool(info.getPoolName(),
+        info.getOwnerName(), info.getGroupName(),
+        info.getMode(), info.getLimit(), info.getMaxRelativeExpiryMs());
+  }
+
+  CachePool(String poolName, String ownerName, String groupName,
+      FsPermission mode, long limit, long maxRelativeExpiry) {
+    Preconditions.checkNotNull(poolName);
+    Preconditions.checkNotNull(ownerName);
+    Preconditions.checkNotNull(groupName);
+    Preconditions.checkNotNull(mode);
+    this.poolName = poolName;
+    this.ownerName = ownerName;
+    this.groupName = groupName;
+    this.mode = new FsPermission(mode);
+    this.limit = limit;
+    this.maxRelativeExpiryMs = maxRelativeExpiry;
+  }
+
+  public String getPoolName() {
+    return poolName;
+  }
+
+  public String getOwnerName() {
+    return ownerName;
+  }
+
+  public CachePool setOwnerName(String ownerName) {
+    this.ownerName = ownerName;
+    return this;
+  }
+
+  public String getGroupName() {
+    return groupName;
+  }
+
+  public CachePool setGroupName(String groupName) {
+    this.groupName = groupName;
+    return this;
+  }
+
+  public FsPermission getMode() {
+    return mode;
+  }
+
+  public CachePool setMode(FsPermission mode) {
+    this.mode = new FsPermission(mode);
+    return this;
+  }
+
+  public long getLimit() {
+    return limit;
+  }
+
+  public CachePool setLimit(long bytes) {
+    this.limit = bytes;
+    return this;
+  }
+
+  public long getMaxRelativeExpiryMs() {
+    return maxRelativeExpiryMs;
+  }
+
+  public CachePool setMaxRelativeExpiryMs(long expiry) {
+    this.maxRelativeExpiryMs = expiry;
+    return this;
+  }
+
+  /**
+   * Get either full or partial information about this CachePool.
+   *
+   * @param fullInfo
+   *          If true, only the name will be returned (i.e., what you 
+   *          would get if you didn't have read permission for this pool.)
+   * @return
+   *          Cache pool information.
+   */
+  CachePoolInfo getInfo(boolean fullInfo) {
+    CachePoolInfo info = new CachePoolInfo(poolName);
+    if (!fullInfo) {
+      return info;
+    }
+    return info.setOwnerName(ownerName).
+        setGroupName(groupName).
+        setMode(new FsPermission(mode)).
+        setLimit(limit).
+        setMaxRelativeExpiryMs(maxRelativeExpiryMs);
+  }
+
+  /**
+   * Resets statistics related to this CachePool
+   */
+  public void resetStatistics() {
+    bytesNeeded = 0;
+    bytesCached = 0;
+    filesNeeded = 0;
+    filesCached = 0;
+  }
+
+  public void addBytesNeeded(long bytes) {
+    bytesNeeded += bytes;
+  }
+
+  public void addBytesCached(long bytes) {
+    bytesCached += bytes;
+  }
+
+  public void addFilesNeeded(long files) {
+    filesNeeded += files;
+  }
+
+  public void addFilesCached(long files) {
+    filesCached += files;
+  }
+
+  public long getBytesNeeded() {
+    return bytesNeeded;
+  }
+
+  public long getBytesCached() {
+    return bytesCached;
+  }
+
+  public long getBytesOverlimit() {
+    return Math.max(bytesNeeded-limit, 0);
+  }
+
+  public long getFilesNeeded() {
+    return filesNeeded;
+  }
+
+  public long getFilesCached() {
+    return filesCached;
+  }
+
+  /**
+   * Get statistics about this CachePool.
+   *
+   * @return   Cache pool statistics.
+   */
+  private CachePoolStats getStats() {
+    return new CachePoolStats.Builder().
+        setBytesNeeded(bytesNeeded).
+        setBytesCached(bytesCached).
+        setBytesOverlimit(getBytesOverlimit()).
+        setFilesNeeded(filesNeeded).
+        setFilesCached(filesCached).
+        build();
+  }
+
+  /**
+   * Returns a CachePoolInfo describing this CachePool based on the permissions
+   * of the calling user. Unprivileged users will see only minimal descriptive
+   * information about the pool.
+   * 
+   * @param pc Permission checker to be used to validate the user's permissions,
+   *          or null
+   * @return CachePoolEntry describing this CachePool
+   */
+  public CachePoolEntry getEntry(FSPermissionChecker pc) {
+    boolean hasPermission = true;
+    if (pc != null) {
+      try {
+        pc.checkPermission(this, FsAction.READ);
+      } catch (AccessControlException e) {
+        hasPermission = false;
+      }
+    }
+    return new CachePoolEntry(getInfo(hasPermission), 
+        hasPermission ? getStats() : new CachePoolStats.Builder().build());
+  }
+
+  public String toString() {
+    return new StringBuilder().
+        append("{ ").append("poolName:").append(poolName).
+        append(", ownerName:").append(ownerName).
+        append(", groupName:").append(groupName).
+        append(", mode:").append(mode).
+        append(", limit:").append(limit).
+        append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
+        append(" }").toString();
+  }
+
+  public DirectiveList getDirectiveList() {
+    return directiveList;
+  }
+}

+ 251 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachedBlock.java

@@ -0,0 +1,251 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
+import org.apache.hadoop.util.IntrusiveCollection;
+import org.apache.hadoop.util.LightWeightGSet;
+import org.apache.hadoop.util.IntrusiveCollection.Element;
+import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
+
+/**
+ * Represents a cached block.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+public final class CachedBlock implements Element, 
+    LightWeightGSet.LinkedElement {
+  private static final Object[] EMPTY_ARRAY = new Object[0];
+
+  /**
+   * Block id.
+   */
+  private final long blockId;
+
+  /**
+   * Used to implement #{LightWeightGSet.LinkedElement}
+   */
+  private LinkedElement nextElement;
+
+  /**
+   * Bit 15: Mark
+   * Bit 0-14: cache replication factor.
+   */
+  private short replicationAndMark;
+
+  /**
+   * Used to implement the CachedBlocksList.
+   *
+   * Since this CachedBlock can be in multiple CachedBlocksList objects,
+   * we need to be able to store multiple 'prev' and 'next' pointers.
+   * The triplets array does this.
+   *
+   * Each triplet contains a CachedBlockList object followed by a
+   * prev pointer, followed by a next pointer.
+   */
+  private Object[] triplets;
+
+  public CachedBlock(long blockId, short replication, boolean mark) {
+    this.blockId = blockId;
+    this.triplets = EMPTY_ARRAY;
+    setReplicationAndMark(replication, mark);
+  }
+
+  public long getBlockId() {
+    return blockId;
+  }
+
+  @Override
+  public int hashCode() {
+    return (int)(blockId^(blockId>>>32));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o == null) { return false; }
+    if (o == this) { return true; }
+    if (o.getClass() != this.getClass()) {
+      return false;
+    }
+    CachedBlock other = (CachedBlock)o;
+    return other.blockId == blockId;
+  }
+
+  public void setReplicationAndMark(short replication, boolean mark) {
+    assert replication >= 0;
+    replicationAndMark = (short)((replication << 1) | (mark ? 0x1 : 0x0));
+  }
+
+  public boolean getMark() {
+    return ((replicationAndMark & 0x1) != 0);
+  }
+
+  public short getReplication() {
+    return (short) (replicationAndMark >>> 1);
+  }
+
+  /**
+   * Return true if this CachedBlock is present on the given list.
+   */
+  public boolean isPresent(CachedBlocksList cachedBlocksList) {
+    for (int i = 0; i < triplets.length; i += 3) {
+      CachedBlocksList list = (CachedBlocksList)triplets[i];
+      if (list == cachedBlocksList) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Get a list of the datanodes which this block is cached,
+   * planned to be cached, or planned to be uncached on.
+   *
+   * @param type      If null, this parameter is ignored.
+   *                  If it is non-null, we match only datanodes which
+   *                  have it on this list.
+   *                  See {@link DatanodeDescriptor#CachedBlocksList#Type}
+   *                  for a description of all the lists.
+   *                  
+   * @return          The list of datanodes.  Modifying this list does not
+   *                  alter the state of the CachedBlock.
+   */
+  public List<DatanodeDescriptor> getDatanodes(Type type) {
+    List<DatanodeDescriptor> nodes = new LinkedList<DatanodeDescriptor>();
+    for (int i = 0; i < triplets.length; i += 3) {
+      CachedBlocksList list = (CachedBlocksList)triplets[i];
+      if ((type == null) || (list.getType() == type)) {
+        nodes.add(list.getDatanode());
+      }
+    }
+    return nodes;
+  }
+
+  @Override
+  public void insertInternal(IntrusiveCollection<? extends Element> list, Element prev,
+      Element next) {
+    for (int i = 0; i < triplets.length; i += 3) {
+      if (triplets[i] == list) {
+        throw new RuntimeException("Trying to re-insert an element that " +
+            "is already in the list.");
+      }
+    }
+    Object newTriplets[] = Arrays.copyOf(triplets, triplets.length + 3);
+    newTriplets[triplets.length] = list;
+    newTriplets[triplets.length + 1] = prev;
+    newTriplets[triplets.length + 2] = next;
+    triplets = newTriplets;
+  }
+  
+  @Override
+  public void setPrev(IntrusiveCollection<? extends Element> list, Element prev) {
+    for (int i = 0; i < triplets.length; i += 3) {
+      if (triplets[i] == list) {
+        triplets[i + 1] = prev;
+        return;
+      }
+    }
+    throw new RuntimeException("Called setPrev on an element that wasn't " +
+        "in the list.");
+  }
+
+  @Override
+  public void setNext(IntrusiveCollection<? extends Element> list, Element next) {
+    for (int i = 0; i < triplets.length; i += 3) {
+      if (triplets[i] == list) {
+        triplets[i + 2] = next;
+        return;
+      }
+    }
+    throw new RuntimeException("Called setNext on an element that wasn't " +
+        "in the list.");
+  }
+
+  @Override
+  public void removeInternal(IntrusiveCollection<? extends Element> list) {
+    for (int i = 0; i < triplets.length; i += 3) {
+      if (triplets[i] == list) {
+        Object[] newTriplets = new Object[triplets.length - 3];
+        System.arraycopy(triplets, 0, newTriplets, 0, i);
+        System.arraycopy(triplets, i + 3, newTriplets, i,
+            triplets.length - (i + 3));
+        triplets = newTriplets;
+        return;
+      }
+    }
+    throw new RuntimeException("Called remove on an element that wasn't " +
+        "in the list.");
+  }
+
+  @Override
+  public Element getPrev(IntrusiveCollection<? extends Element> list) {
+    for (int i = 0; i < triplets.length; i += 3) {
+      if (triplets[i] == list) {
+        return (Element)triplets[i + 1];
+      }
+    }
+    throw new RuntimeException("Called getPrev on an element that wasn't " +
+        "in the list.");
+  }
+
+  @Override
+  public Element getNext(IntrusiveCollection<? extends Element> list) {
+    for (int i = 0; i < triplets.length; i += 3) {
+      if (triplets[i] == list) {
+        return (Element)triplets[i + 2];
+      }
+    }
+    throw new RuntimeException("Called getNext on an element that wasn't " +
+        "in the list.");
+  }
+
+  @Override
+  public boolean isInList(IntrusiveCollection<? extends Element> list) {
+    for (int i = 0; i < triplets.length; i += 3) {
+      if (triplets[i] == list) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  @Override
+  public String toString() {
+    return new StringBuilder().append("{").
+        append("blockId=").append(blockId).append(", ").
+        append("replication=").append(getReplication()).append(", ").
+        append("mark=").append(getMark()).append("}").
+        toString();
+  }
+
+  @Override // LightWeightGSet.LinkedElement 
+  public void setNext(LinkedElement next) {
+    this.nextElement = next;
+  }
+
+  @Override // LightWeightGSet.LinkedElement 
+  public LinkedElement getNext() {
+    return nextElement;
+  }
+}

+ 16 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongExce
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
@@ -2641,12 +2642,21 @@ public class FSDirectory implements Closeable {
     int childrenNum = node.isDirectory() ? 
     int childrenNum = node.isDirectory() ? 
         node.asDirectory().getChildrenNum(snapshot) : 0;
         node.asDirectory().getChildrenNum(snapshot) : 0;
         
         
-    return new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
-        blocksize, node.getModificationTime(snapshot),
-        node.getAccessTime(snapshot), node.getFsPermission(snapshot),
-        node.getUserName(snapshot), node.getGroupName(snapshot),
-        node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
-        node.getId(), loc, childrenNum);
+    HdfsLocatedFileStatus status =
+        new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
+          blocksize, node.getModificationTime(snapshot),
+          node.getAccessTime(snapshot), node.getFsPermission(snapshot),
+          node.getUserName(snapshot), node.getGroupName(snapshot),
+          node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
+          node.getId(), loc, childrenNum);
+        // Set caching information for the located blocks.
+    if (loc != null) {
+      CacheManager cacheManager = namesystem.getCacheManager();
+      for (LocatedBlock lb: loc.getLocatedBlocks()) {
+        cacheManager.setCachedLocations(lb);
+      }
+    }
+    return status;
   }
   }
 
 
     
     

+ 63 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -17,6 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import static org.apache.hadoop.util.ExitUtil.terminate;
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.now;
 
 
 import java.net.URI;
 import java.net.URI;
@@ -34,17 +35,19 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 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.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 
 
-import static org.apache.hadoop.util.ExitUtil.terminate;
-
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
@@ -57,12 +60,18 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogSegmentOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogSegmentOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCacheDirectiveInfoOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV1Op;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV2Op;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
@@ -71,8 +80,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV1Op;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV2Op;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
@@ -955,7 +962,57 @@ public class FSEditLog implements LogsPurgeable {
         .setSnapshotRoot(path);
         .setSnapshotRoot(path);
     logEdit(op);
     logEdit(op);
   }
   }
-  
+
+  /**
+   * Log a CacheDirectiveInfo returned from
+   * {@link CacheManager#addDirective(CacheDirectiveInfo, FSPermissionChecker)}
+   */
+  void logAddCacheDirectiveInfo(CacheDirectiveInfo directive,
+      boolean toLogRpcIds) {
+    AddCacheDirectiveInfoOp op =
+        AddCacheDirectiveInfoOp.getInstance(cache.get())
+            .setDirective(directive);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
+
+  void logModifyCacheDirectiveInfo(
+      CacheDirectiveInfo directive, boolean toLogRpcIds) {
+    ModifyCacheDirectiveInfoOp op =
+        ModifyCacheDirectiveInfoOp.getInstance(
+            cache.get()).setDirective(directive);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
+
+  void logRemoveCacheDirectiveInfo(Long id, boolean toLogRpcIds) {
+    RemoveCacheDirectiveInfoOp op =
+        RemoveCacheDirectiveInfoOp.getInstance(cache.get()).setId(id);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
+
+  void logAddCachePool(CachePoolInfo pool, boolean toLogRpcIds) {
+    AddCachePoolOp op =
+        AddCachePoolOp.getInstance(cache.get()).setPool(pool);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
+
+  void logModifyCachePool(CachePoolInfo info, boolean toLogRpcIds) {
+    ModifyCachePoolOp op =
+        ModifyCachePoolOp.getInstance(cache.get()).setInfo(info);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
+
+  void logRemoveCachePool(String poolName, boolean toLogRpcIds) {
+    RemoveCachePoolOp op =
+        RemoveCachePoolOp.getInstance(cache.get()).setPoolName(poolName);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
+
   /**
   /**
    * Get all the journals this edit log is currently operating on.
    * Get all the journals this edit log is currently operating on.
    */
    */

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

@@ -24,6 +24,7 @@ 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;
@@ -31,17 +32,21 @@ 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.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 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.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
@@ -54,7 +59,11 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
@@ -78,6 +87,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Co
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.Holder;
 import org.apache.hadoop.hdfs.util.Holder;
+import org.apache.jasper.tagplugins.jstl.core.Remove;
 
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -648,6 +658,59 @@ public class FSEditLogLoader {
       fsNamesys.setLastAllocatedBlockId(allocateBlockIdOp.blockId);
       fsNamesys.setLastAllocatedBlockId(allocateBlockIdOp.blockId);
       break;
       break;
     }
     }
+    case OP_ADD_CACHE_DIRECTIVE: {
+      AddCacheDirectiveInfoOp addOp = (AddCacheDirectiveInfoOp) op;
+      CacheDirectiveInfo result = fsNamesys.
+          getCacheManager().addDirectiveFromEditLog(addOp.directive);
+      if (toAddRetryCache) {
+        Long id = result.getId();
+        fsNamesys.addCacheEntryWithPayload(op.rpcClientId, op.rpcCallId, id);
+      }
+      break;
+    }
+    case OP_MODIFY_CACHE_DIRECTIVE: {
+      ModifyCacheDirectiveInfoOp modifyOp =
+          (ModifyCacheDirectiveInfoOp) op;
+      fsNamesys.getCacheManager().modifyDirectiveFromEditLog(
+          modifyOp.directive);
+      if (toAddRetryCache) {
+        fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
+      }
+      break;
+    }
+    case OP_REMOVE_CACHE_DIRECTIVE: {
+      RemoveCacheDirectiveInfoOp removeOp =
+          (RemoveCacheDirectiveInfoOp) op;
+      fsNamesys.getCacheManager().removeDirective(removeOp.id, null);
+      if (toAddRetryCache) {
+        fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
+      }
+      break;
+    }
+    case OP_ADD_CACHE_POOL: {
+      AddCachePoolOp addOp = (AddCachePoolOp) op;
+      fsNamesys.getCacheManager().addCachePool(addOp.info);
+      if (toAddRetryCache) {
+        fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
+      }
+      break;
+    }
+    case OP_MODIFY_CACHE_POOL: {
+      ModifyCachePoolOp modifyOp = (ModifyCachePoolOp) op;
+      fsNamesys.getCacheManager().modifyCachePool(modifyOp.info);
+      if (toAddRetryCache) {
+        fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
+      }
+      break;
+    }
+    case OP_REMOVE_CACHE_POOL: {
+      RemoveCachePoolOp removeOp = (RemoveCachePoolOp) op;
+      fsNamesys.getCacheManager().removeCachePool(removeOp.poolName);
+      if (toAddRetryCache) {
+        fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
+      }
+      break;
+    }
     default:
     default:
       throw new IOException("Invalid operation read " + op.opCode);
       throw new IOException("Invalid operation read " + op.opCode);
     }
     }

+ 414 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_BLOCK;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_BLOCK;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_DIRECTIVE;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_POOL;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOCATE_BLOCK_ID;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOCATE_BLOCK_ID;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOW_SNAPSHOT;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOW_SNAPSHOT;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CANCEL_DELEGATION_TOKEN;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CANCEL_DELEGATION_TOKEN;
@@ -33,7 +35,11 @@ import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_END_LOG
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_GET_DELEGATION_TOKEN;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_GET_DELEGATION_TOKEN;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_INVALID;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_INVALID;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MKDIR;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MKDIR;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MODIFY_CACHE_DIRECTIVE;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MODIFY_CACHE_POOL;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REASSIGN_LEASE;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REASSIGN_LEASE;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_CACHE_DIRECTIVE;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_CACHE_POOL;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_OLD;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_OLD;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_SNAPSHOT;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_SNAPSHOT;
@@ -57,6 +63,7 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.EnumMap;
 import java.util.EnumMap;
 import java.util.List;
 import java.util.List;
@@ -74,6 +81,8 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 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.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
@@ -98,6 +107,7 @@ import org.xml.sax.ContentHandler;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXException;
 import org.xml.sax.helpers.AttributesImpl;
 import org.xml.sax.helpers.AttributesImpl;
 
 
+import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 
 
 /**
 /**
@@ -151,6 +161,15 @@ public abstract class FSEditLogOp {
       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_BLOCK, new AddBlockOp());
       inst.put(OP_ADD_BLOCK, new AddBlockOp());
+      inst.put(OP_ADD_CACHE_DIRECTIVE,
+          new AddCacheDirectiveInfoOp());
+      inst.put(OP_MODIFY_CACHE_DIRECTIVE,
+          new ModifyCacheDirectiveInfoOp());
+      inst.put(OP_REMOVE_CACHE_DIRECTIVE,
+          new RemoveCacheDirectiveInfoOp());
+      inst.put(OP_ADD_CACHE_POOL, new AddCachePoolOp());
+      inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
+      inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
     }
     }
     
     
     public FSEditLogOp get(FSEditLogOpCodes opcode) {
     public FSEditLogOp get(FSEditLogOpCodes opcode) {
@@ -525,8 +544,7 @@ public abstract class FSEditLogOp {
       } else {
       } else {
         this.blocks = new Block[0];
         this.blocks = new Block[0];
       }
       }
-      this.permissions =
-          permissionStatusFromXml(st.getChildren("PERMISSION_STATUS").get(0));
+      this.permissions = permissionStatusFromXml(st);
       readRpcIdsFromXml(st);
       readRpcIdsFromXml(st);
     }
     }
   }
   }
@@ -1307,8 +1325,7 @@ public abstract class FSEditLogOp {
       this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.path = st.getValue("PATH");
       this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
       this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
-      this.permissions =
-          permissionStatusFromXml(st.getChildren("PERMISSION_STATUS").get(0));
+      this.permissions = permissionStatusFromXml(st);
     }
     }
   }
   }
 
 
@@ -2039,8 +2056,7 @@ public abstract class FSEditLogOp {
       this.value = st.getValue("VALUE");
       this.value = st.getValue("VALUE");
       this.mtime = Long.valueOf(st.getValue("MTIME"));
       this.mtime = Long.valueOf(st.getValue("MTIME"));
       this.atime = Long.valueOf(st.getValue("ATIME"));
       this.atime = Long.valueOf(st.getValue("ATIME"));
-      this.permissionStatus =
-          permissionStatusFromXml(st.getChildren("PERMISSION_STATUS").get(0));
+      this.permissionStatus = permissionStatusFromXml(st);
       
       
       readRpcIdsFromXml(st);
       readRpcIdsFromXml(st);
     }
     }
@@ -2947,6 +2963,381 @@ public abstract class FSEditLogOp {
     }
     }
   }
   }
 
 
+  /**
+   * {@literal @AtMostOnce} for
+   * {@link ClientProtocol#addCacheDirective}
+   */
+  static class AddCacheDirectiveInfoOp extends FSEditLogOp {
+    CacheDirectiveInfo directive;
+
+    public AddCacheDirectiveInfoOp() {
+      super(OP_ADD_CACHE_DIRECTIVE);
+    }
+
+    static AddCacheDirectiveInfoOp getInstance(OpInstanceCache cache) {
+      return (AddCacheDirectiveInfoOp) cache
+          .get(OP_ADD_CACHE_DIRECTIVE);
+    }
+
+    public AddCacheDirectiveInfoOp setDirective(
+        CacheDirectiveInfo directive) {
+      this.directive = directive;
+      assert(directive.getId() != null);
+      assert(directive.getPath() != null);
+      assert(directive.getReplication() != null);
+      assert(directive.getPool() != null);
+      assert(directive.getExpiration() != null);
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      directive = FSImageSerialization.readCacheDirectiveInfo(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeCacheDirectiveInfo(out, directive);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      FSImageSerialization.writeCacheDirectiveInfo(contentHandler, directive);
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      directive = FSImageSerialization.readCacheDirectiveInfo(st);
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("AddCacheDirectiveInfo [");
+      builder.append("id=" + directive.getId() + ",");
+      builder.append("path=" + directive.getPath().toUri().getPath() + ",");
+      builder.append("replication=" + directive.getReplication() + ",");
+      builder.append("pool=" + directive.getPool() + ",");
+      builder.append("expiration=" + directive.getExpiration().getMillis());
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /**
+   * {@literal @AtMostOnce} for
+   * {@link ClientProtocol#modifyCacheDirective}
+   */
+  static class ModifyCacheDirectiveInfoOp extends FSEditLogOp {
+    CacheDirectiveInfo directive;
+
+    public ModifyCacheDirectiveInfoOp() {
+      super(OP_MODIFY_CACHE_DIRECTIVE);
+    }
+
+    static ModifyCacheDirectiveInfoOp getInstance(OpInstanceCache cache) {
+      return (ModifyCacheDirectiveInfoOp) cache
+          .get(OP_MODIFY_CACHE_DIRECTIVE);
+    }
+
+    public ModifyCacheDirectiveInfoOp setDirective(
+        CacheDirectiveInfo directive) {
+      this.directive = directive;
+      assert(directive.getId() != null);
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      this.directive = FSImageSerialization.readCacheDirectiveInfo(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeCacheDirectiveInfo(out, directive);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      FSImageSerialization.writeCacheDirectiveInfo(contentHandler, directive);
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.directive = FSImageSerialization.readCacheDirectiveInfo(st);
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("ModifyCacheDirectiveInfoOp[");
+      builder.append("id=").append(directive.getId());
+      if (directive.getPath() != null) {
+        builder.append(",").append("path=").append(directive.getPath());
+      }
+      if (directive.getReplication() != null) {
+        builder.append(",").append("replication=").
+            append(directive.getReplication());
+      }
+      if (directive.getPool() != null) {
+        builder.append(",").append("pool=").append(directive.getPool());
+      }
+      if (directive.getExpiration() != null) {
+        builder.append(",").append("expiration=").
+            append(directive.getExpiration().getMillis());
+      }
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /**
+   * {@literal @AtMostOnce} for
+   * {@link ClientProtocol#removeCacheDirective}
+   */
+  static class RemoveCacheDirectiveInfoOp extends FSEditLogOp {
+    long id;
+
+    public RemoveCacheDirectiveInfoOp() {
+      super(OP_REMOVE_CACHE_DIRECTIVE);
+    }
+
+    static RemoveCacheDirectiveInfoOp getInstance(OpInstanceCache cache) {
+      return (RemoveCacheDirectiveInfoOp) cache
+          .get(OP_REMOVE_CACHE_DIRECTIVE);
+    }
+
+    public RemoveCacheDirectiveInfoOp setId(long id) {
+      this.id = id;
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      this.id = FSImageSerialization.readLong(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(id, out);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "ID", Long.toString(id));
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.id = Long.parseLong(st.getValue("ID"));
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("RemoveCacheDirectiveInfo [");
+      builder.append("id=" + Long.toString(id));
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /** {@literal @AtMostOnce} for {@link ClientProtocol#addCachePool} */
+  static class AddCachePoolOp extends FSEditLogOp {
+    CachePoolInfo info;
+
+    public AddCachePoolOp() {
+      super(OP_ADD_CACHE_POOL);
+    }
+
+    static AddCachePoolOp getInstance(OpInstanceCache cache) {
+      return (AddCachePoolOp) cache.get(OP_ADD_CACHE_POOL);
+    }
+
+    public AddCachePoolOp setPool(CachePoolInfo info) {
+      this.info = info;
+      assert(info.getPoolName() != null);
+      assert(info.getOwnerName() != null);
+      assert(info.getGroupName() != null);
+      assert(info.getMode() != null);
+      assert(info.getLimit() != null);
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      info = FSImageSerialization.readCachePoolInfo(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeCachePoolInfo(out, info);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      FSImageSerialization.writeCachePoolInfo(contentHandler, info);
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.info = FSImageSerialization.readCachePoolInfo(st);
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("AddCachePoolOp [");
+      builder.append("poolName=" + info.getPoolName() + ",");
+      builder.append("ownerName=" + info.getOwnerName() + ",");
+      builder.append("groupName=" + info.getGroupName() + ",");
+      builder.append("mode=" + Short.toString(info.getMode().toShort()) + ",");
+      builder.append("limit=" + Long.toString(info.getLimit()));
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /** {@literal @AtMostOnce} for {@link ClientProtocol#modifyCachePool} */
+  static class ModifyCachePoolOp extends FSEditLogOp {
+    CachePoolInfo info;
+
+    public ModifyCachePoolOp() {
+      super(OP_MODIFY_CACHE_POOL);
+    }
+
+    static ModifyCachePoolOp getInstance(OpInstanceCache cache) {
+      return (ModifyCachePoolOp) cache.get(OP_MODIFY_CACHE_POOL);
+    }
+
+    public ModifyCachePoolOp setInfo(CachePoolInfo info) {
+      this.info = info;
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      info = FSImageSerialization.readCachePoolInfo(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeCachePoolInfo(out, info);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      FSImageSerialization.writeCachePoolInfo(contentHandler, info);
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.info = FSImageSerialization.readCachePoolInfo(st);
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("ModifyCachePoolOp [");
+      ArrayList<String> fields = new ArrayList<String>(5);
+      if (info.getPoolName() != null) {
+        fields.add("poolName=" + info.getPoolName());
+      }
+      if (info.getOwnerName() != null) {
+        fields.add("ownerName=" + info.getOwnerName());
+      }
+      if (info.getGroupName() != null) {
+        fields.add("groupName=" + info.getGroupName());
+      }
+      if (info.getMode() != null) {
+        fields.add("mode=" + info.getMode().toString());
+      }
+      if (info.getLimit() != null) {
+        fields.add("limit=" + info.getLimit());
+      }
+      builder.append(Joiner.on(",").join(fields));
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /** {@literal @AtMostOnce} for {@link ClientProtocol#removeCachePool} */
+  static class RemoveCachePoolOp extends FSEditLogOp {
+    String poolName;
+
+    public RemoveCachePoolOp() {
+      super(OP_REMOVE_CACHE_POOL);
+    }
+
+    static RemoveCachePoolOp getInstance(OpInstanceCache cache) {
+      return (RemoveCachePoolOp) cache.get(OP_REMOVE_CACHE_POOL);
+    }
+
+    public RemoveCachePoolOp setPoolName(String poolName) {
+      this.poolName = poolName;
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      poolName = FSImageSerialization.readString(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(poolName, out);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "POOLNAME", poolName);
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.poolName = st.getValue("POOLNAME");
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("RemoveCachePoolOp [");
+      builder.append("poolName=" + poolName);
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
   static private short readShort(DataInputStream in) throws IOException {
   static private short readShort(DataInputStream in) throws IOException {
     return Short.parseShort(FSImageSerialization.readString(in));
     return Short.parseShort(FSImageSerialization.readString(in));
   }
   }
@@ -3332,16 +3723,28 @@ public abstract class FSEditLogOp {
     contentHandler.startElement("", "", "PERMISSION_STATUS", new AttributesImpl());
     contentHandler.startElement("", "", "PERMISSION_STATUS", new AttributesImpl());
     XMLUtils.addSaxString(contentHandler, "USERNAME", perm.getUserName());
     XMLUtils.addSaxString(contentHandler, "USERNAME", perm.getUserName());
     XMLUtils.addSaxString(contentHandler, "GROUPNAME", perm.getGroupName());
     XMLUtils.addSaxString(contentHandler, "GROUPNAME", perm.getGroupName());
-    XMLUtils.addSaxString(contentHandler, "MODE",
-        Short.valueOf(perm.getPermission().toShort()).toString());
+    fsPermissionToXml(contentHandler, perm.getPermission());
     contentHandler.endElement("", "", "PERMISSION_STATUS");
     contentHandler.endElement("", "", "PERMISSION_STATUS");
   }
   }
 
 
   public static PermissionStatus permissionStatusFromXml(Stanza st)
   public static PermissionStatus permissionStatusFromXml(Stanza st)
       throws InvalidXmlException {
       throws InvalidXmlException {
-    String username = st.getValue("USERNAME");
-    String groupname = st.getValue("GROUPNAME");
+    Stanza status = st.getChildren("PERMISSION_STATUS").get(0);
+    String username = status.getValue("USERNAME");
+    String groupname = status.getValue("GROUPNAME");
+    FsPermission mode = fsPermissionFromXml(status);
+    return new PermissionStatus(username, groupname, mode);
+  }
+
+  public static void fsPermissionToXml(ContentHandler contentHandler,
+      FsPermission mode) throws SAXException {
+    XMLUtils.addSaxString(contentHandler, "MODE", Short.valueOf(mode.toShort())
+        .toString());
+  }
+
+  public static FsPermission fsPermissionFromXml(Stanza st)
+      throws InvalidXmlException {
     short mode = Short.valueOf(st.getValue("MODE"));
     short mode = Short.valueOf(st.getValue("MODE"));
-    return new PermissionStatus(username, groupname, new FsPermission(mode));
+    return new FsPermission(mode);
   }
   }
 }
 }

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java

@@ -61,6 +61,12 @@ public enum FSEditLogOpCodes {
   OP_SET_GENSTAMP_V2            ((byte) 31),
   OP_SET_GENSTAMP_V2            ((byte) 31),
   OP_ALLOCATE_BLOCK_ID          ((byte) 32),
   OP_ALLOCATE_BLOCK_ID          ((byte) 32),
   OP_ADD_BLOCK                  ((byte) 33),
   OP_ADD_BLOCK                  ((byte) 33),
+  OP_ADD_CACHE_DIRECTIVE       ((byte) 34),
+  OP_REMOVE_CACHE_DIRECTIVE    ((byte) 35),
+  OP_ADD_CACHE_POOL                       ((byte) 36),
+  OP_MODIFY_CACHE_POOL                    ((byte) 37),
+  OP_REMOVE_CACHE_POOL                    ((byte) 38),
+  OP_MODIFY_CACHE_DIRECTIVE    ((byte) 39),
 
 
   // Note that fromByte(..) depends on OP_INVALID being at the last position.  
   // Note that fromByte(..) depends on OP_INVALID being at the last position.  
   OP_INVALID                    ((byte) -1);
   OP_INVALID                    ((byte) -1);

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

@@ -358,6 +358,8 @@ public class FSImageFormat {
 
 
         loadSecretManagerState(in);
         loadSecretManagerState(in);
 
 
+        loadCacheManagerState(in);
+
         // make sure to read to the end of file
         // make sure to read to the end of file
         boolean eof = (in.read() == -1);
         boolean eof = (in.read() == -1);
         assert eof : "Should have reached the end of image file " + curFile;
         assert eof : "Should have reached the end of image file " + curFile;
@@ -897,6 +899,14 @@ public class FSImageFormat {
       namesystem.loadSecretManagerState(in);
       namesystem.loadSecretManagerState(in);
     }
     }
 
 
+    private void loadCacheManagerState(DataInput in) throws IOException {
+      int imgVersion = getLayoutVersion();
+      if (!LayoutVersion.supports(Feature.CACHING, imgVersion)) {
+        return;
+      }
+      namesystem.getCacheManager().loadState(in);
+    }
+
     private int getLayoutVersion() {
     private int getLayoutVersion() {
       return namesystem.getFSImage().getStorage().getLayoutVersion();
       return namesystem.getFSImage().getStorage().getLayoutVersion();
     }
     }
@@ -1051,6 +1061,8 @@ public class FSImageFormat {
         context.checkCancelled();
         context.checkCancelled();
         sourceNamesystem.saveSecretManagerState(out, sdPath);
         sourceNamesystem.saveSecretManagerState(out, sdPath);
         context.checkCancelled();
         context.checkCancelled();
+        sourceNamesystem.getCacheManager().saveState(out, sdPath);
+        context.checkCancelled();
         out.flush();
         out.flush();
         context.checkCancelled();
         context.checkCancelled();
         fout.getChannel().force(true);
         fout.getChannel().force(true);

+ 224 - 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;
@@ -39,11 +41,16 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottab
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 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;
 
 
@@ -480,4 +487,221 @@ 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();
+    final Long maxRelativeExpiry = info.getMaxRelativeExpiryMs();
+
+    boolean hasOwner, hasGroup, hasMode, hasLimit, hasMaxRelativeExpiry;
+    hasOwner = ownerName != null;
+    hasGroup = groupName != null;
+    hasMode = mode != null;
+    hasLimit = limit != null;
+    hasMaxRelativeExpiry = maxRelativeExpiry != null;
+
+    int flags =
+        (hasOwner ? 0x1 : 0) |
+        (hasGroup ? 0x2 : 0) |
+        (hasMode  ? 0x4 : 0) |
+        (hasLimit ? 0x8 : 0) |
+        (hasMaxRelativeExpiry ? 0x10 : 0);
+
+    writeInt(flags, out);
+
+    if (hasOwner) {
+      writeString(ownerName, out);
+    }
+    if (hasGroup) {
+      writeString(groupName, out);
+    }
+    if (hasMode) {
+      mode.write(out);
+    }
+    if (hasLimit) {
+      writeLong(limit, out);
+    }
+    if (hasMaxRelativeExpiry) {
+      writeLong(maxRelativeExpiry, 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 & 0x10) != 0) {
+      info.setMaxRelativeExpiryMs(readLong(in));
+    }
+    if ((flags & ~0x1F) != 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();
+    final Long maxRelativeExpiry = info.getMaxRelativeExpiryMs();
+
+    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));
+    }
+    if (maxRelativeExpiry != null) {
+      XMLUtils.addSaxString(contentHandler, "MAXRELATIVEEXPIRY",
+          Long.toString(maxRelativeExpiry));
+    }
+  }
+
+  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")));
+    }
+    if (st.hasChildren("MAXRELATIVEEXPIRY")) {
+      info.setMaxRelativeExpiryMs(
+          Long.parseLong(st.getValue("MAXRELATIVEEXPIRY")));
+    }
+    return info;
+  }
+
 }
 }

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

@@ -125,6 +125,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 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.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;
@@ -150,6 +152,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -161,6 +165,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -363,6 +369,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   FSDirectory dir;
   FSDirectory dir;
   private final BlockManager blockManager;
   private final BlockManager blockManager;
   private final SnapshotManager snapshotManager;
   private final SnapshotManager snapshotManager;
+  private final CacheManager cacheManager;
   private final DatanodeStatistics datanodeStatistics;
   private final DatanodeStatistics datanodeStatistics;
 
 
   // Block pool ID used by this namenode
   // Block pool ID used by this namenode
@@ -716,6 +723,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       this.dtSecretManager = createDelegationTokenSecretManager(conf);
       this.dtSecretManager = createDelegationTokenSecretManager(conf);
       this.dir = new FSDirectory(fsImage, this, conf);
       this.dir = new FSDirectory(fsImage, this, conf);
       this.snapshotManager = new SnapshotManager(dir);
       this.snapshotManager = new SnapshotManager(dir);
+      this.cacheManager = new CacheManager(this, conf, blockManager);
       this.safeMode = new SafeModeInfo(conf);
       this.safeMode = new SafeModeInfo(conf);
       this.auditLoggers = initAuditLoggers(conf);
       this.auditLoggers = initAuditLoggers(conf);
       this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
       this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
@@ -939,7 +947,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         blockManager.getDatanodeManager().markAllDatanodesStale();
         blockManager.getDatanodeManager().markAllDatanodesStale();
         blockManager.clearQueues();
         blockManager.clearQueues();
         blockManager.processAllPendingDNMessages();
         blockManager.processAllPendingDNMessages();
-        
+
         if (!isInSafeMode() ||
         if (!isInSafeMode() ||
             (isInSafeMode() && safeMode.isPopulatingReplQueues())) {
             (isInSafeMode() && safeMode.isPopulatingReplQueues())) {
           LOG.info("Reprocessing replication and invalidation queues");
           LOG.info("Reprocessing replication and invalidation queues");
@@ -977,6 +985,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           editLogRollerThreshold, editLogRollerInterval));
           editLogRollerThreshold, editLogRollerInterval));
       nnEditLogRoller.start();
       nnEditLogRoller.start();
 
 
+      cacheManager.startMonitorThread();
+      blockManager.getDatanodeManager().setShouldSendCachingCommands(true);
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
       startingActiveService = false;
       startingActiveService = false;
@@ -1026,6 +1036,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         // so that the tailer starts from the right spot.
         // so that the tailer starts from the right spot.
         dir.fsImage.updateLastAppliedTxIdFromWritten();
         dir.fsImage.updateLastAppliedTxIdFromWritten();
       }
       }
+      cacheManager.stopMonitorThread();
+      cacheManager.clearDirectiveStats();
+      blockManager.getDatanodeManager().clearPendingCachingCommands();
+      blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
     }
     }
@@ -1602,8 +1616,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           length = Math.min(length, fileSize - offset);
           length = Math.min(length, fileSize - offset);
           isUc = false;
           isUc = false;
         }
         }
-        return blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
+        LocatedBlocks blocks =
+          blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
             isUc, offset, length, needBlockToken, iip.isSnapshot());
             isUc, offset, length, needBlockToken, iip.isSnapshot());
+        // Set caching information for the located blocks.
+        for (LocatedBlock lb: blocks.getLocatedBlocks()) {
+          cacheManager.setCachedLocations(lb);
+        }
+        return blocks;
       } finally {
       } finally {
         if (isReadOp) {
         if (isReadOp) {
           readUnlock();
           readUnlock();
@@ -4108,15 +4128,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException
    * @throws IOException
    */
    */
   HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
   HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
-      StorageReport[] reports, int xceiverCount, int xmitsInProgress,
-      int failedVolumes)
+      StorageReport[] reports, long cacheCapacity, long cacheUsed,
+      int xceiverCount, int xmitsInProgress, int failedVolumes)
         throws IOException {
         throws IOException {
     readLock();
     readLock();
     try {
     try {
       final int maxTransfer = blockManager.getMaxReplicationStreams()
       final int maxTransfer = blockManager.getMaxReplicationStreams()
           - xmitsInProgress;
           - xmitsInProgress;
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
-          nodeReg, reports, blockPoolId, xceiverCount, maxTransfer, failedVolumes);
+          nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
+          xceiverCount, maxTransfer, failedVolumes);
       return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
       return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
     } finally {
     } finally {
       readUnlock();
       readUnlock();
@@ -6391,6 +6412,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return datanodeStatistics.getCapacityRemainingPercent();
     return datanodeStatistics.getCapacityRemainingPercent();
   }
   }
 
 
+  @Override // NameNodeMXBean
+  public long getCacheCapacity() {
+    return datanodeStatistics.getCacheCapacity();
+  }
+
+  @Override // NameNodeMXBean
+  public long getCacheUsed() {
+    return datanodeStatistics.getCacheUsed();
+  }
+
   @Override // NameNodeMXBean
   @Override // NameNodeMXBean
   public long getTotalBlocks() {
   public long getTotalBlocks() {
     return getBlocksTotal();
     return getBlocksTotal();
@@ -6639,6 +6670,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public FSDirectory getFSDirectory() {
   public FSDirectory getFSDirectory() {
     return dir;
     return dir;
   }
   }
+  /** @return the cache manager. */
+  public CacheManager getCacheManager() {
+    return cacheManager;
+  }
 
 
   @Override  // NameNodeMXBean
   @Override  // NameNodeMXBean
   public String getCorruptFiles() {
   public String getCorruptFiles() {
@@ -7016,6 +7051,262 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     }
   }
   }
 
 
+  long addCacheDirective(CacheDirectiveInfo directive, EnumSet<CacheFlag> flags)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = isPermissionEnabled ?
+        getPermissionChecker() : null;
+    CacheEntryWithPayload cacheEntry =
+        RetryCache.waitForCompletion(retryCache, null);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return (Long) cacheEntry.getPayload();
+    }
+    boolean success = false;
+    if (!flags.contains(CacheFlag.FORCE)) {
+      cacheManager.waitForRescanIfNeeded();
+    }
+    writeLock();
+    Long result = null;
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException(
+            "Cannot add cache directive", safeMode);
+      }
+      if (directive.getId() != null) {
+        throw new IOException("addDirective: you cannot specify an ID " +
+            "for this operation.");
+      }
+      CacheDirectiveInfo effectiveDirective = 
+          cacheManager.addDirective(directive, pc, flags);
+      getEditLog().logAddCacheDirectiveInfo(effectiveDirective,
+          cacheEntry != null);
+      result = effectiveDirective.getId();
+      success = true;
+    } finally {
+      writeUnlock();
+      if (success) {
+        getEditLog().logSync();
+      }
+      if (isAuditEnabled() && isExternalInvocation()) {
+        logAuditEvent(success, "addCacheDirective", null, null, null);
+      }
+      RetryCache.setState(cacheEntry, success, result);
+    }
+    return result;
+  }
+
+  void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = isPermissionEnabled ?
+        getPermissionChecker() : null;
+    boolean success = false;
+    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return;
+    }
+    if (!flags.contains(CacheFlag.FORCE)) {
+      cacheManager.waitForRescanIfNeeded();
+    }
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException(
+            "Cannot add cache directive", safeMode);
+      }
+      cacheManager.modifyDirective(directive, pc, flags);
+      getEditLog().logModifyCacheDirectiveInfo(directive,
+          cacheEntry != null);
+      success = true;
+    } finally {
+      writeUnlock();
+      if (success) {
+        getEditLog().logSync();
+      }
+      if (isAuditEnabled() && isExternalInvocation()) {
+        logAuditEvent(success, "modifyCacheDirective", null, null, null);
+      }
+      RetryCache.setState(cacheEntry, success);
+    }
+  }
+
+  void removeCacheDirective(Long id) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = isPermissionEnabled ?
+        getPermissionChecker() : null;
+    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return;
+    }
+    boolean success = false;
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException(
+            "Cannot remove cache directives", safeMode);
+      }
+      cacheManager.removeDirective(id, pc);
+      getEditLog().logRemoveCacheDirectiveInfo(id, cacheEntry != null);
+      success = true;
+    } finally {
+      writeUnlock();
+      if (isAuditEnabled() && isExternalInvocation()) {
+        logAuditEvent(success, "removeCacheDirective", null, null,
+            null);
+      }
+      RetryCache.setState(cacheEntry, success);
+    }
+    getEditLog().logSync();
+  }
+
+  BatchedListEntries<CacheDirectiveEntry> listCacheDirectives(
+      long startId, CacheDirectiveInfo filter) throws IOException {
+    checkOperation(OperationCategory.READ);
+    final FSPermissionChecker pc = isPermissionEnabled ?
+        getPermissionChecker() : null;
+    BatchedListEntries<CacheDirectiveEntry> results;
+    cacheManager.waitForRescanIfNeeded();
+    readLock();
+    boolean success = false;
+    try {
+      checkOperation(OperationCategory.READ);
+      results =
+          cacheManager.listCacheDirectives(startId, filter, pc);
+      success = true;
+    } finally {
+      readUnlock();
+      if (isAuditEnabled() && isExternalInvocation()) {
+        logAuditEvent(success, "listCacheDirectives", null, null,
+            null);
+      }
+    }
+    return results;
+  }
+
+  public void addCachePool(CachePoolInfo req) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = isPermissionEnabled ?
+        getPermissionChecker() : null;
+    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return; // Return previous response
+    }
+    writeLock();
+    boolean success = false;
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException(
+            "Cannot add cache pool " + req.getPoolName(), safeMode);
+      }
+      if (pc != null) {
+        pc.checkSuperuserPrivilege();
+      }
+      CachePoolInfo info = cacheManager.addCachePool(req);
+      getEditLog().logAddCachePool(info, cacheEntry != null);
+      success = true;
+    } finally {
+      writeUnlock();
+      if (isAuditEnabled() && isExternalInvocation()) {
+        logAuditEvent(success, "addCachePool", req.getPoolName(), null, null);
+      }
+      RetryCache.setState(cacheEntry, success);
+    }
+    
+    getEditLog().logSync();
+  }
+
+  public void modifyCachePool(CachePoolInfo req) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc =
+        isPermissionEnabled ? getPermissionChecker() : null;
+    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return; // Return previous response
+    }
+    writeLock();
+    boolean success = false;
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException(
+            "Cannot modify cache pool " + req.getPoolName(), safeMode);
+      }
+      if (pc != null) {
+        pc.checkSuperuserPrivilege();
+      }
+      cacheManager.modifyCachePool(req);
+      getEditLog().logModifyCachePool(req, cacheEntry != null);
+      success = true;
+    } finally {
+      writeUnlock();
+      if (isAuditEnabled() && isExternalInvocation()) {
+        logAuditEvent(success, "modifyCachePool", req.getPoolName(), null, null);
+      }
+      RetryCache.setState(cacheEntry, success);
+    }
+
+    getEditLog().logSync();
+  }
+
+  public void removeCachePool(String cachePoolName) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc =
+        isPermissionEnabled ? getPermissionChecker() : null;
+    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return; // Return previous response
+    }
+    writeLock();
+    boolean success = false;
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException(
+            "Cannot remove cache pool " + cachePoolName, safeMode);
+      }
+      if (pc != null) {
+        pc.checkSuperuserPrivilege();
+      }
+      cacheManager.removeCachePool(cachePoolName);
+      getEditLog().logRemoveCachePool(cachePoolName, cacheEntry != null);
+      success = true;
+    } finally {
+      writeUnlock();
+      if (isAuditEnabled() && isExternalInvocation()) {
+        logAuditEvent(success, "removeCachePool", cachePoolName, null, null);
+      }
+      RetryCache.setState(cacheEntry, success);
+    }
+    
+    getEditLog().logSync();
+  }
+
+  public BatchedListEntries<CachePoolEntry> listCachePools(String prevKey)
+      throws IOException {
+    final FSPermissionChecker pc =
+        isPermissionEnabled ? getPermissionChecker() : null;
+    BatchedListEntries<CachePoolEntry> results;
+    checkOperation(OperationCategory.READ);
+    boolean success = false;
+    cacheManager.waitForRescanIfNeeded();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      results = cacheManager.listCachePools(pc, prevKey);
+      success = true;
+    } finally {
+      readUnlock();
+      if (isAuditEnabled() && isExternalInvocation()) {
+        logAuditEvent(success, "listCachePools", null, null, null);
+      }
+    }
+    return results;
+  }
+
   /**
   /**
    * Default AuditLogger implementation; used when no access logger is
    * Default AuditLogger implementation; used when no access logger is
    * defined in the config file. It can also be explicitly listed in the
    * defined in the config file. It can also be explicitly listed in the
@@ -7078,6 +7369,5 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       auditLog.info(message);
       auditLog.info(message);
     }
     }
   }
   }
-
 }
 }
 
 

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java

@@ -255,4 +255,33 @@ class FSPermissionChecker {
     throw new AccessControlException("Permission denied by sticky bit setting:" +
     throw new AccessControlException("Permission denied by sticky bit setting:" +
       " user=" + user + ", inode=" + inode);
       " user=" + user + ", inode=" + inode);
   }
   }
+
+  /**
+   * Whether a cache pool can be accessed by the current context
+   *
+   * @param pool CachePool being accessed
+   * @param access type of action being performed on the cache pool
+   * @throws AccessControlException if pool cannot be accessed
+   */
+  public void checkPermission(CachePool pool, FsAction access)
+      throws AccessControlException {
+    FsPermission mode = pool.getMode();
+    if (isSuperUser()) {
+      return;
+    }
+    if (user.equals(pool.getOwnerName())
+        && mode.getUserAction().implies(access)) {
+      return;
+    }
+    if (groups.contains(pool.getGroupName())
+        && mode.getGroupAction().implies(access)) {
+      return;
+    }
+    if (mode.getOtherAction().implies(access)) {
+      return;
+    }
+    throw new AccessControlException("Permission denied while accessing pool "
+        + pool.getPoolName() + ": user " + user + " does not have "
+        + access.toString() + " permissions.");
+  }
 }
 }

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -656,8 +656,13 @@ public class NameNode implements NameNodeStatusMXBean {
     try {
     try {
       initializeGenericKeys(conf, nsId, namenodeId);
       initializeGenericKeys(conf, nsId, namenodeId);
       initialize(conf);
       initialize(conf);
-      state.prepareToEnterState(haContext);
-      state.enterState(haContext);
+      try {
+        haContext.writeLock();
+        state.prepareToEnterState(haContext);
+        state.enterState(haContext);
+      } finally {
+        haContext.writeUnlock();
+      }
     } catch (IOException e) {
     } catch (IOException e) {
       this.stop();
       this.stop();
       throw e;
       throw e;

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java

@@ -101,6 +101,16 @@ public interface NameNodeMXBean {
    * @return the percentage of the remaining space on the cluster
    * @return the percentage of the remaining space on the cluster
    */
    */
   public float getPercentRemaining();
   public float getPercentRemaining();
+
+  /**
+   * Returns the amount of cache used by the datanode (in bytes).
+   */
+  public long getCacheUsed();
+
+  /**
+   * Returns the total cache capacity of the datanode (in bytes).
+   */
+  public long getCacheCapacity();
   
   
   /**
   /**
    * Get the total space used by the block pools of this namenode
    * Get the total space used by the block pools of this namenode

+ 70 - 2
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;
@@ -46,6 +48,7 @@ import org.apache.hadoop.fs.Options;
 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.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 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.ha.HAServiceStatus;
 import org.apache.hadoop.ha.HAServiceStatus;
@@ -60,6 +63,10 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -953,11 +960,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
   @Override // DatanodeProtocol
   @Override // DatanodeProtocol
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration nodeReg,
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration nodeReg,
-      StorageReport[] report, int xmitsInProgress, int xceiverCount,
+      StorageReport[] report, long dnCacheCapacity, long dnCacheUsed,
+      int xmitsInProgress, int xceiverCount,
       int failedVolumes) throws IOException {
       int failedVolumes) throws IOException {
     verifyRequest(nodeReg);
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, report,
     return namesystem.handleHeartbeat(nodeReg, report,
-        xceiverCount, xmitsInProgress, failedVolumes);
+        dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
+        failedVolumes);
   }
   }
 
 
   @Override // DatanodeProtocol
   @Override // DatanodeProtocol
@@ -979,6 +988,18 @@ class NameNodeRpcServer implements NamenodeProtocols {
     return null;
     return null;
   }
   }
 
 
+  @Override
+  public DatanodeCommand cacheReport(DatanodeRegistration nodeReg,
+      String poolId, List<Long> blockIds) throws IOException {
+    verifyRequest(nodeReg);
+    if (blockStateChangeLog.isDebugEnabled()) {
+      blockStateChangeLog.debug("*BLOCK* NameNode.cacheReport: "
+           + "from " + nodeReg + " " + blockIds.size() + " blocks");
+    }
+    namesystem.getCacheManager().processCacheReport(nodeReg, blockIds);
+    return null;
+  }
+
   @Override // DatanodeProtocol
   @Override // DatanodeProtocol
   public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
   public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
       StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
       StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
@@ -1214,5 +1235,52 @@ class NameNodeRpcServer implements NamenodeProtocols {
     metrics.incrSnapshotDiffReportOps();
     metrics.incrSnapshotDiffReportOps();
     return report;
     return report;
   }
   }
+
+  @Override
+  public long addCacheDirective(
+      CacheDirectiveInfo path, EnumSet<CacheFlag> flags) throws IOException {
+    return namesystem.addCacheDirective(path, flags);
+  }
+
+  @Override
+  public void modifyCacheDirective(
+      CacheDirectiveInfo directive, EnumSet<CacheFlag> flags) throws IOException {
+    namesystem.modifyCacheDirective(directive, flags);
+  }
+
+  @Override
+  public void removeCacheDirective(long id) throws IOException {
+    namesystem.removeCacheDirective(id);
+  }
+
+  @Override
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(long prevId,
+      CacheDirectiveInfo filter) throws IOException {
+    if (filter == null) {
+      filter = new CacheDirectiveInfo.Builder().build();
+    }
+    return namesystem.listCacheDirectives(prevId, filter);
+  }
+
+  @Override
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    namesystem.addCachePool(info);
+  }
+
+  @Override
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    namesystem.modifyCachePool(info);
+  }
+
+  @Override
+  public void removeCachePool(String cachePoolName) throws IOException {
+    namesystem.removeCachePool(cachePoolName);
+  }
+
+  @Override
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+      throws IOException {
+    return namesystem.listCachePools(prevKey != null ? prevKey : "");
+  }
 }
 }
 
 

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

@@ -81,6 +81,8 @@ public class NameNodeMetrics {
   MutableCounterLong transactionsBatchedInSync;
   MutableCounterLong transactionsBatchedInSync;
   @Metric("Block report") MutableRate blockReport;
   @Metric("Block report") MutableRate blockReport;
   MutableQuantiles[] blockReportQuantiles;
   MutableQuantiles[] blockReportQuantiles;
+  @Metric("Cache report") MutableRate cacheReport;
+  MutableQuantiles[] cacheReportQuantiles;
 
 
   @Metric("Duration in SafeMode at startup in msec")
   @Metric("Duration in SafeMode at startup in msec")
   MutableGaugeInt safeModeTime;
   MutableGaugeInt safeModeTime;
@@ -100,6 +102,7 @@ public class NameNodeMetrics {
     final int len = intervals.length;
     final int len = intervals.length;
     syncsQuantiles = new MutableQuantiles[len];
     syncsQuantiles = new MutableQuantiles[len];
     blockReportQuantiles = new MutableQuantiles[len];
     blockReportQuantiles = new MutableQuantiles[len];
+    cacheReportQuantiles = new MutableQuantiles[len];
     
     
     for (int i = 0; i < len; i++) {
     for (int i = 0; i < len; i++) {
       int interval = intervals[i];
       int interval = intervals[i];
@@ -109,6 +112,9 @@ public class NameNodeMetrics {
       blockReportQuantiles[i] = registry.newQuantiles(
       blockReportQuantiles[i] = registry.newQuantiles(
           "blockReport" + interval + "s", 
           "blockReport" + interval + "s", 
           "Block report", "ops", "latency", interval);
           "Block report", "ops", "latency", interval);
+      cacheReportQuantiles[i] = registry.newQuantiles(
+          "cacheReport" + interval + "s",
+          "Cache report", "ops", "latency", interval);
     }
     }
   }
   }
 
 
@@ -242,6 +248,13 @@ public class NameNodeMetrics {
     }
     }
   }
   }
 
 
+  public void addCacheBlockReport(long latency) {
+    cacheReport.add(latency);
+    for (MutableQuantiles q : cacheReportQuantiles) {
+      q.add(latency);
+    }
+  }
+
   public void setSafeModeTime(long elapsed) {
   public void setSafeModeTime(long elapsed) {
     safeModeTime.set((int) elapsed);
     safeModeTime.set((int) elapsed);
   }
   }

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java

@@ -42,7 +42,17 @@ public enum StepType {
   /**
   /**
    * The namenode is performing an operation related to inodes.
    * The namenode is performing an operation related to inodes.
    */
    */
-  INODES("Inodes", "inodes");
+  INODES("Inodes", "inodes"),
+
+  /**
+   * The namenode is performing an operation related to cache pools.
+   */
+  CACHE_POOLS("CachePools", "cache pools"),
+
+  /**
+   * The namenode is performing an operation related to cache entries.
+   */
+  CACHE_ENTRIES("CacheEntries", "cache entries");
 
 
   private final String name, description;
   private final String name, description;
 
 

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockIdCommand.java

@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/****************************************************
+ * A BlockIdCommand is an instruction to a datanode 
+ * regarding some blocks under its control.
+ ****************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockIdCommand extends DatanodeCommand {
+  final String poolId;
+  final long blockIds[];
+
+  /**
+   * Create BlockCommand for the given action
+   * @param blocks blocks related to the action
+   */
+  public BlockIdCommand(int action, String poolId, long[] blockIds) {
+    super(action);
+    this.poolId = poolId;
+    this.blockIds= blockIds;
+  }
+  
+  public String getBlockPoolId() {
+    return poolId;
+  }
+  
+  public long[] getBlockIds() {
+    return blockIds;
+  }
+}

+ 24 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java

@@ -19,13 +19,14 @@
 package org.apache.hadoop.hdfs.server.protocol;
 package org.apache.hadoop.hdfs.server.protocol;
 
 
 import java.io.*;
 import java.io.*;
+import java.util.List;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 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.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.io.retry.AtMostOnce;
 import org.apache.hadoop.io.retry.Idempotent;
 import org.apache.hadoop.io.retry.Idempotent;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.KerberosInfo;
 
 
@@ -74,6 +75,8 @@ public interface DatanodeProtocol {
   final static int DNA_RECOVERBLOCK = 6;  // request a block recovery
   final static int DNA_RECOVERBLOCK = 6;  // request a block recovery
   final static int DNA_ACCESSKEYUPDATE = 7;  // update access key
   final static int DNA_ACCESSKEYUPDATE = 7;  // update access key
   final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth
   final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth
+  final static int DNA_CACHE = 9;      // cache blocks
+  final static int DNA_UNCACHE = 10;   // uncache blocks
 
 
   /** 
   /** 
    * Register Datanode.
    * Register Datanode.
@@ -104,6 +107,8 @@ public interface DatanodeProtocol {
   @Idempotent
   @Idempotent
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
                                        StorageReport[] reports,
                                        StorageReport[] reports,
+                                       long dnCacheCapacity,
+                                       long dnCacheUsed,
                                        int xmitsInProgress,
                                        int xmitsInProgress,
                                        int xceiverCount,
                                        int xceiverCount,
                                        int failedVolumes) throws IOException;
                                        int failedVolumes) throws IOException;
@@ -128,6 +133,24 @@ public interface DatanodeProtocol {
   public DatanodeCommand blockReport(DatanodeRegistration registration,
   public DatanodeCommand blockReport(DatanodeRegistration registration,
       String poolId, StorageBlockReport[] reports) throws IOException;
       String poolId, StorageBlockReport[] reports) throws IOException;
     
     
+
+  /**
+   * Communicates the complete list of locally cached blocks to the NameNode.
+   * 
+   * This method is similar to
+   * {@link #blockReport(DatanodeRegistration, String, StorageBlockReport[])},
+   * which is used to communicated blocks stored on disk.
+   *
+   * @param            The datanode registration.
+   * @param poolId     The block pool ID for the blocks.
+   * @param blockIds   A list of block IDs.
+   * @return           The DatanodeCommand.
+   * @throws IOException
+   */
+  @Idempotent
+  public DatanodeCommand cacheReport(DatanodeRegistration registration,
+      String poolId, List<Long> blockIds) throws IOException;
+
   /**
   /**
    * blockReceivedAndDeleted() allows the DataNode to tell the NameNode about
    * blockReceivedAndDeleted() allows the DataNode to tell the NameNode about
    * recently-received and -deleted block data. 
    * recently-received and -deleted block data. 

+ 1059 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java

@@ -0,0 +1,1059 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.lang.WordUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolStats;
+import org.apache.hadoop.hdfs.tools.TableListing.Justification;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+
+import com.google.common.base.Joiner;
+
+/**
+ * This class implements command-line operations on the HDFS Cache.
+ */
+@InterfaceAudience.Private
+public class CacheAdmin extends Configured implements Tool {
+
+  /**
+   * Maximum length for printed lines
+   */
+  private static final int MAX_LINE_WIDTH = 80;
+
+  public CacheAdmin() {
+    this(null);
+  }
+
+  public CacheAdmin(Configuration conf) {
+    super(conf);
+  }
+
+  @Override
+  public int run(String[] args) throws IOException {
+    if (args.length == 0) {
+      printUsage(false);
+      return 1;
+    }
+    Command command = determineCommand(args[0]);
+    if (command == null) {
+      System.err.println("Can't understand command '" + args[0] + "'");
+      if (!args[0].startsWith("-")) {
+        System.err.println("Command names must start with dashes.");
+      }
+      printUsage(false);
+      return 1;
+    }
+    List<String> argsList = new LinkedList<String>();
+    for (int j = 1; j < args.length; j++) {
+      argsList.add(args[j]);
+    }
+    try {
+      return command.run(getConf(), argsList);
+    } catch (IllegalArgumentException e) {
+      System.err.println(prettifyException(e));
+      return -1;
+    }
+  }
+
+  public static void main(String[] argsArray) throws IOException {
+    CacheAdmin cacheAdmin = new CacheAdmin(new Configuration());
+    System.exit(cacheAdmin.run(argsArray));
+  }
+
+  private static DistributedFileSystem getDFS(Configuration conf)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    if (!(fs instanceof DistributedFileSystem)) {
+      throw new IllegalArgumentException("FileSystem " + fs.getUri() + 
+      " is not an HDFS file system");
+    }
+    return (DistributedFileSystem)fs;
+  }
+
+  /**
+   * NN exceptions contain the stack trace as part of the exception message.
+   * When it's a known error, pretty-print the error and squish the stack trace.
+   */
+  private static String prettifyException(Exception e) {
+    return e.getClass().getSimpleName() + ": "
+        + e.getLocalizedMessage().split("\n")[0];
+  }
+
+  private static TableListing getOptionDescriptionListing() {
+    TableListing listing = new TableListing.Builder()
+    .addField("").addField("", true)
+    .wrapWidth(MAX_LINE_WIDTH).hideHeaders().build();
+    return listing;
+  }
+
+  /**
+   * Parses a time-to-live value from a string
+   * @return The ttl in milliseconds
+   * @throws IOException if it could not be parsed
+   */
+  private static Long parseTtlString(String maxTtlString) throws IOException {
+    Long maxTtl = null;
+    if (maxTtlString != null) {
+      if (maxTtlString.equalsIgnoreCase("never")) {
+        maxTtl = CachePoolInfo.RELATIVE_EXPIRY_NEVER;
+      } else {
+        maxTtl = DFSUtil.parseRelativeTime(maxTtlString);
+      }
+    }
+    return maxTtl;
+  }
+
+  private static Expiration parseExpirationString(String ttlString)
+      throws IOException {
+    Expiration ex = null;
+    if (ttlString != null) {
+      if (ttlString.equalsIgnoreCase("never")) {
+        ex = CacheDirectiveInfo.Expiration.NEVER;
+      } else {
+        long ttl = DFSUtil.parseRelativeTime(ttlString);
+        ex = CacheDirectiveInfo.Expiration.newRelative(ttl);
+      }
+    }
+    return ex;
+  }
+
+  interface Command {
+    String getName();
+    String getShortUsage();
+    String getLongUsage();
+    int run(Configuration conf, List<String> args) throws IOException;
+  }
+
+  private static class AddCacheDirectiveInfoCommand implements Command {
+    @Override
+    public String getName() {
+      return "-addDirective";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() +
+          " -path <path> -pool <pool-name> " +
+          "[-force] " +
+          "[-replication <replication>] [-ttl <time-to-live>]]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+      listing.addRow("<path>", "A path to cache. The path can be " +
+          "a directory or a file.");
+      listing.addRow("<pool-name>", "The pool to which the directive will be " +
+          "added. You must have write permission on the cache pool "
+          + "in order to add new directives.");
+      listing.addRow("-force",
+          "Skips checking of cache pool resource limits.");
+      listing.addRow("<replication>", "The cache replication factor to use. " +
+          "Defaults to 1.");
+      listing.addRow("<time-to-live>", "How long the directive is " +
+          "valid. Can be specified in minutes, hours, and days, e.g. " +
+          "30m, 4h, 2d. Valid units are [smhd]." +
+          " \"never\" indicates a directive that never expires." +
+          " If unspecified, the directive never expires.");
+      return getShortUsage() + "\n" +
+        "Add a new cache directive.\n\n" +
+        listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder();
+
+      String path = StringUtils.popOptionWithArgument("-path", args);
+      if (path == null) {
+        System.err.println("You must specify a path with -path.");
+        return 1;
+      }
+      builder.setPath(new Path(path));
+
+      String poolName = StringUtils.popOptionWithArgument("-pool", args);
+      if (poolName == null) {
+        System.err.println("You must specify a pool name with -pool.");
+        return 1;
+      }
+      builder.setPool(poolName);
+      boolean force = StringUtils.popOption("-force", args);
+      String replicationString =
+          StringUtils.popOptionWithArgument("-replication", args);
+      if (replicationString != null) {
+        Short replication = Short.parseShort(replicationString);
+        builder.setReplication(replication);
+      }
+
+      String ttlString = StringUtils.popOptionWithArgument("-ttl", args);
+      try {
+        Expiration ex = parseExpirationString(ttlString);
+        if (ex != null) {
+          builder.setExpiration(ex);
+        }
+      } catch (IOException e) {
+        System.err.println(
+            "Error while parsing ttl value: " + e.getMessage());
+        return 1;
+      }
+
+      if (!args.isEmpty()) {
+        System.err.println("Can't understand argument: " + args.get(0));
+        return 1;
+      }
+        
+      DistributedFileSystem dfs = getDFS(conf);
+      CacheDirectiveInfo directive = builder.build();
+      EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
+      if (force) {
+        flags.add(CacheFlag.FORCE);
+      }
+      try {
+        long id = dfs.addCacheDirective(directive, flags);
+        System.out.println("Added cache directive " + id);
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+
+      return 0;
+    }
+  }
+
+  private static class RemoveCacheDirectiveInfoCommand implements Command {
+    @Override
+    public String getName() {
+      return "-removeDirective";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " <id>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+      listing.addRow("<id>", "The id of the cache directive to remove.  " + 
+        "You must have write permission on the pool of the " +
+        "directive in order to remove it.  To see a list " +
+        "of cache directive IDs, use the -listDirectives command.");
+      return getShortUsage() + "\n" +
+        "Remove a cache directive.\n\n" +
+        listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      String idString= StringUtils.popFirstNonOption(args);
+      if (idString == null) {
+        System.err.println("You must specify a directive ID to remove.");
+        return 1;
+      }
+      long id;
+      try {
+        id = Long.valueOf(idString);
+      } catch (NumberFormatException e) {
+        System.err.println("Invalid directive ID " + idString + ": expected " +
+            "a numeric value.");
+        return 1;
+      }
+      if (id <= 0) {
+        System.err.println("Invalid directive ID " + id + ": ids must " +
+            "be greater than 0.");
+        return 1;
+      }
+      if (!args.isEmpty()) {
+        System.err.println("Can't understand argument: " + args.get(0));
+        System.err.println("Usage is " + getShortUsage());
+        return 1;
+      }
+      DistributedFileSystem dfs = getDFS(conf);
+      try {
+        dfs.getClient().removeCacheDirective(id);
+        System.out.println("Removed cached directive " + id);
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
+  private static class ModifyCacheDirectiveInfoCommand implements Command {
+    @Override
+    public String getName() {
+      return "-modifyDirective";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() +
+          " -id <id> [-path <path>] [-force] [-replication <replication>] " +
+          "[-pool <pool-name>] [-ttl <time-to-live>]]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+      listing.addRow("<id>", "The ID of the directive to modify (required)");
+      listing.addRow("<path>", "A path to cache. The path can be " +
+          "a directory or a file. (optional)");
+      listing.addRow("-force",
+          "Skips checking of cache pool resource limits.");
+      listing.addRow("<replication>", "The cache replication factor to use. " +
+          "(optional)");
+      listing.addRow("<pool-name>", "The pool to which the directive will be " +
+          "added. You must have write permission on the cache pool "
+          + "in order to move a directive into it. (optional)");
+      listing.addRow("<time-to-live>", "How long the directive is " +
+          "valid. Can be specified in minutes, hours, and days, e.g. " +
+          "30m, 4h, 2d. Valid units are [smhd]." +
+          " \"never\" indicates a directive that never expires.");
+      return getShortUsage() + "\n" +
+        "Modify a cache directive.\n\n" +
+        listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      CacheDirectiveInfo.Builder builder =
+        new CacheDirectiveInfo.Builder();
+      boolean modified = false;
+      String idString = StringUtils.popOptionWithArgument("-id", args);
+      if (idString == null) {
+        System.err.println("You must specify a directive ID with -id.");
+        return 1;
+      }
+      builder.setId(Long.parseLong(idString));
+      String path = StringUtils.popOptionWithArgument("-path", args);
+      if (path != null) {
+        builder.setPath(new Path(path));
+        modified = true;
+      }
+      boolean force = StringUtils.popOption("-force", args);
+      String replicationString =
+        StringUtils.popOptionWithArgument("-replication", args);
+      if (replicationString != null) {
+        builder.setReplication(Short.parseShort(replicationString));
+        modified = true;
+      }
+      String poolName =
+        StringUtils.popOptionWithArgument("-pool", args);
+      if (poolName != null) {
+        builder.setPool(poolName);
+        modified = true;
+      }
+      String ttlString = StringUtils.popOptionWithArgument("-ttl", args);
+      try {
+        Expiration ex = parseExpirationString(ttlString);
+        if (ex != null) {
+          builder.setExpiration(ex);
+          modified = true;
+        }
+      } catch (IOException e) {
+        System.err.println(
+            "Error while parsing ttl value: " + e.getMessage());
+        return 1;
+      }
+      if (!args.isEmpty()) {
+        System.err.println("Can't understand argument: " + args.get(0));
+        System.err.println("Usage is " + getShortUsage());
+        return 1;
+      }
+      if (!modified) {
+        System.err.println("No modifications were specified.");
+        return 1;
+      }
+      DistributedFileSystem dfs = getDFS(conf);
+      EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
+      if (force) {
+        flags.add(CacheFlag.FORCE);
+      }
+      try {
+        dfs.modifyCacheDirective(builder.build(), flags);
+        System.out.println("Modified cache directive " + idString);
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
+  private static class RemoveCacheDirectiveInfosCommand implements Command {
+    @Override
+    public String getName() {
+      return "-removeDirectives";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " -path <path>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+      listing.addRow("-path <path>", "The path of the cache directives to remove.  " +
+        "You must have write permission on the pool of the directive in order " +
+        "to remove it.  To see a list of cache directives, use the " +
+        "-listDirectives command.");
+      return getShortUsage() + "\n" +
+        "Remove every cache directive with the specified path.\n\n" +
+        listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      String path = StringUtils.popOptionWithArgument("-path", args);
+      if (path == null) {
+        System.err.println("You must specify a path with -path.");
+        return 1;
+      }
+      if (!args.isEmpty()) {
+        System.err.println("Can't understand argument: " + args.get(0));
+        System.err.println("Usage is " + getShortUsage());
+        return 1;
+      }
+      int exitCode = 0;
+      try {
+        DistributedFileSystem dfs = getDFS(conf);
+        RemoteIterator<CacheDirectiveEntry> iter =
+            dfs.listCacheDirectives(
+                new CacheDirectiveInfo.Builder().
+                    setPath(new Path(path)).build());
+        while (iter.hasNext()) {
+          CacheDirectiveEntry entry = iter.next();
+          try {
+            dfs.removeCacheDirective(entry.getInfo().getId());
+            System.out.println("Removed cache directive " +
+                entry.getInfo().getId());
+          } catch (IOException e) {
+            System.err.println(prettifyException(e));
+            exitCode = 2;
+          }
+        }
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        exitCode = 2;
+      }
+      if (exitCode == 0) {
+        System.out.println("Removed every cache directive with path " +
+            path);
+      }
+      return exitCode;
+    }
+  }
+
+  private static class ListCacheDirectiveInfoCommand implements Command {
+    @Override
+    public String getName() {
+      return "-listDirectives";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " [-stats] [-path <path>] [-pool <pool>]]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+      listing.addRow("<path>", "List only " +
+          "cache directives with this path. " +
+          "Note that if there is a cache directive for <path> " +
+          "in a cache pool that we don't have read access for, it " + 
+          "will not be listed.");
+      listing.addRow("<pool>", "List only path cache directives in that pool.");
+      listing.addRow("-stats", "List path-based cache directive statistics.");
+      return getShortUsage() + "\n" +
+        "List cache directives.\n\n" +
+        listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      CacheDirectiveInfo.Builder builder =
+          new CacheDirectiveInfo.Builder();
+      String pathFilter = StringUtils.popOptionWithArgument("-path", args);
+      if (pathFilter != null) {
+        builder.setPath(new Path(pathFilter));
+      }
+      String poolFilter = StringUtils.popOptionWithArgument("-pool", args);
+      if (poolFilter != null) {
+        builder.setPool(poolFilter);
+      }
+      boolean printStats = StringUtils.popOption("-stats", args);
+      if (!args.isEmpty()) {
+        System.err.println("Can't understand argument: " + args.get(0));
+        return 1;
+      }
+      TableListing.Builder tableBuilder = new TableListing.Builder().
+          addField("ID", Justification.RIGHT).
+          addField("POOL", Justification.LEFT).
+          addField("REPL", Justification.RIGHT).
+          addField("EXPIRY", Justification.LEFT).
+          addField("PATH", Justification.LEFT);
+      if (printStats) {
+        tableBuilder.addField("BYTES_NEEDED", Justification.RIGHT).
+                    addField("BYTES_CACHED", Justification.RIGHT).
+                    addField("FILES_NEEDED", Justification.RIGHT).
+                    addField("FILES_CACHED", Justification.RIGHT);
+      }
+      TableListing tableListing = tableBuilder.build();
+      try {
+        DistributedFileSystem dfs = getDFS(conf);
+        RemoteIterator<CacheDirectiveEntry> iter =
+            dfs.listCacheDirectives(builder.build());
+        int numEntries = 0;
+        while (iter.hasNext()) {
+          CacheDirectiveEntry entry = iter.next();
+          CacheDirectiveInfo directive = entry.getInfo();
+          CacheDirectiveStats stats = entry.getStats();
+          List<String> row = new LinkedList<String>();
+          row.add("" + directive.getId());
+          row.add(directive.getPool());
+          row.add("" + directive.getReplication());
+          String expiry;
+          // This is effectively never, round for nice printing
+          if (directive.getExpiration().getMillis() >
+              Expiration.MAX_RELATIVE_EXPIRY_MS / 2) {
+            expiry = "never";
+          } else {
+            expiry = directive.getExpiration().toString();
+          }
+          row.add(expiry);
+          row.add(directive.getPath().toUri().getPath());
+          if (printStats) {
+            row.add("" + stats.getBytesNeeded());
+            row.add("" + stats.getBytesCached());
+            row.add("" + stats.getFilesNeeded());
+            row.add("" + stats.getFilesCached());
+          }
+          tableListing.addRow(row.toArray(new String[0]));
+          numEntries++;
+        }
+        System.out.print(String.format("Found %d entr%s\n",
+            numEntries, numEntries == 1 ? "y" : "ies"));
+        if (numEntries > 0) {
+          System.out.print(tableListing);
+        }
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
+  private static class AddCachePoolCommand implements Command {
+
+    private static final String NAME = "-addPool";
+
+    @Override
+    public String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + NAME + " <name> [-owner <owner>] " +
+          "[-group <group>] [-mode <mode>] [-limit <limit>] " +
+          "[-maxTtl <maxTtl>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+
+      listing.addRow("<name>", "Name of the new pool.");
+      listing.addRow("<owner>", "Username of the owner of the pool. " +
+          "Defaults to the current user.");
+      listing.addRow("<group>", "Group of the pool. " +
+          "Defaults to the primary group name of the current user.");
+      listing.addRow("<mode>", "UNIX-style permissions for the pool. " +
+          "Permissions are specified in octal, e.g. 0755. " +
+          "By default, this is set to " + String.format("0%03o",
+          FsPermission.getCachePoolDefault().toShort()) + ".");
+      listing.addRow("<limit>", "The maximum number of bytes that can be " +
+          "cached by directives in this pool, in aggregate. By default, " +
+          "no limit is set.");
+      listing.addRow("<maxTtl>", "The maximum allowed time-to-live for " +
+          "directives being added to the pool. This can be specified in " +
+          "seconds, minutes, hours, and days, e.g. 120s, 30m, 4h, 2d. " +
+          "Valid units are [smhd]. By default, no maximum is set. " +
+          "This can also be manually specified by \"never\".");
+      return getShortUsage() + "\n" +
+          "Add a new cache pool.\n\n" + 
+          listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      String name = StringUtils.popFirstNonOption(args);
+      if (name == null) {
+        System.err.println("You must specify a name when creating a " +
+            "cache pool.");
+        return 1;
+      }
+      CachePoolInfo info = new CachePoolInfo(name);
+
+      String owner = StringUtils.popOptionWithArgument("-owner", args);
+      if (owner != null) {
+        info.setOwnerName(owner);
+      }
+      String group = StringUtils.popOptionWithArgument("-group", args);
+      if (group != null) {
+        info.setGroupName(group);
+      }
+      String modeString = StringUtils.popOptionWithArgument("-mode", args);
+      if (modeString != null) {
+        short mode = Short.parseShort(modeString, 8);
+        info.setMode(new FsPermission(mode));
+      }
+      String limitString = StringUtils.popOptionWithArgument("-limit", args);
+      if (limitString != null) {
+        long limit = Long.parseLong(limitString);
+        info.setLimit(limit);
+      }
+      String maxTtlString = StringUtils.popOptionWithArgument("-maxTtl", args);
+      try {
+        Long maxTtl = parseTtlString(maxTtlString);
+        if (maxTtl != null) {
+          info.setMaxRelativeExpiryMs(maxTtl);
+        }
+      } catch (IOException e) {
+        System.err.println(
+            "Error while parsing maxTtl value: " + e.getMessage());
+        return 1;
+      }
+
+      if (!args.isEmpty()) {
+        System.err.print("Can't understand arguments: " +
+          Joiner.on(" ").join(args) + "\n");
+        System.err.println("Usage is " + getShortUsage());
+        return 1;
+      }
+      DistributedFileSystem dfs = getDFS(conf);
+      try {
+        dfs.addCachePool(info);
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+      System.out.println("Successfully added cache pool " + name + ".");
+      return 0;
+    }
+  }
+
+  private static class ModifyCachePoolCommand implements Command {
+
+    @Override
+    public String getName() {
+      return "-modifyPool";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " <name> [-owner <owner>] " +
+          "[-group <group>] [-mode <mode>] [-limit <limit>] " +
+          "[-maxTtl <maxTtl>]]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+
+      listing.addRow("<name>", "Name of the pool to modify.");
+      listing.addRow("<owner>", "Username of the owner of the pool");
+      listing.addRow("<group>", "Groupname of the group of the pool.");
+      listing.addRow("<mode>", "Unix-style permissions of the pool in octal.");
+      listing.addRow("<limit>", "Maximum number of bytes that can be cached " +
+          "by this pool.");
+      listing.addRow("<maxTtl>", "The maximum allowed time-to-live for " +
+          "directives being added to the pool.");
+
+      return getShortUsage() + "\n" +
+          WordUtils.wrap("Modifies the metadata of an existing cache pool. " +
+          "See usage of " + AddCachePoolCommand.NAME + " for more details.",
+          MAX_LINE_WIDTH) + "\n\n" +
+          listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      String owner = StringUtils.popOptionWithArgument("-owner", args);
+      String group = StringUtils.popOptionWithArgument("-group", args);
+      String modeString = StringUtils.popOptionWithArgument("-mode", args);
+      Integer mode = (modeString == null) ?
+          null : Integer.parseInt(modeString, 8);
+      String limitString = StringUtils.popOptionWithArgument("-limit", args);
+      Long limit = (limitString == null) ?
+          null : Long.parseLong(limitString);
+      String maxTtlString = StringUtils.popOptionWithArgument("-maxTtl", args);
+      Long maxTtl = null;
+      try {
+        maxTtl = parseTtlString(maxTtlString);
+      } catch (IOException e) {
+        System.err.println(
+            "Error while parsing maxTtl value: " + e.getMessage());
+        return 1;
+      }
+      String name = StringUtils.popFirstNonOption(args);
+      if (name == null) {
+        System.err.println("You must specify a name when creating a " +
+            "cache pool.");
+        return 1;
+      }
+      if (!args.isEmpty()) {
+        System.err.print("Can't understand arguments: " +
+          Joiner.on(" ").join(args) + "\n");
+        System.err.println("Usage is " + getShortUsage());
+        return 1;
+      }
+      boolean changed = false;
+      CachePoolInfo info = new CachePoolInfo(name);
+      if (owner != null) {
+        info.setOwnerName(owner);
+        changed = true;
+      }
+      if (group != null) {
+        info.setGroupName(group);
+        changed = true;
+      }
+      if (mode != null) {
+        info.setMode(new FsPermission(mode.shortValue()));
+        changed = true;
+      }
+      if (limit != null) {
+        info.setLimit(limit);
+        changed = true;
+      }
+      if (maxTtl != null) {
+        info.setMaxRelativeExpiryMs(maxTtl);
+        changed = true;
+      }
+      if (!changed) {
+        System.err.println("You must specify at least one attribute to " +
+            "change in the cache pool.");
+        return 1;
+      }
+      DistributedFileSystem dfs = getDFS(conf);
+      try {
+        dfs.modifyCachePool(info);
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+      System.out.print("Successfully modified cache pool " + name);
+      String prefix = " to have ";
+      if (owner != null) {
+        System.out.print(prefix + "owner name " + owner);
+        prefix = " and ";
+      }
+      if (group != null) {
+        System.out.print(prefix + "group name " + group);
+        prefix = " and ";
+      }
+      if (mode != null) {
+        System.out.print(prefix + "mode " + new FsPermission(mode.shortValue()));
+        prefix = " and ";
+      }
+      if (limit != null) {
+        System.out.print(prefix + "limit " + limit);
+        prefix = " and ";
+      }
+      if (maxTtl != null) {
+        System.out.print(prefix + "max time-to-live " + maxTtlString);
+      }
+      System.out.print("\n");
+      return 0;
+    }
+  }
+
+  private static class RemoveCachePoolCommand implements Command {
+
+    @Override
+    public String getName() {
+      return "-removePool";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " <name>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      return getShortUsage() + "\n" +
+          WordUtils.wrap("Remove a cache pool. This also uncaches paths " +
+              "associated with the pool.\n\n", MAX_LINE_WIDTH) +
+          "<name>  Name of the cache pool to remove.\n";
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      String name = StringUtils.popFirstNonOption(args);
+      if (name == null) {
+        System.err.println("You must specify a name when deleting a " +
+            "cache pool.");
+        return 1;
+      }
+      if (!args.isEmpty()) {
+        System.err.print("Can't understand arguments: " +
+          Joiner.on(" ").join(args) + "\n");
+        System.err.println("Usage is " + getShortUsage());
+        return 1;
+      }
+      DistributedFileSystem dfs = getDFS(conf);
+      try {
+        dfs.removeCachePool(name);
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+      System.out.println("Successfully removed cache pool " + name + ".");
+      return 0;
+    }
+  }
+
+  private static class ListCachePoolsCommand implements Command {
+
+    @Override
+    public String getName() {
+      return "-listPools";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " [-stats] [<name>]]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+      listing.addRow("-stats", "Display additional cache pool statistics.");
+      listing.addRow("<name>", "If specified, list only the named cache pool.");
+
+      return getShortUsage() + "\n" +
+          WordUtils.wrap("Display information about one or more cache pools, " +
+              "e.g. name, owner, group, permissions, etc.", MAX_LINE_WIDTH) +
+          "\n\n" +
+          listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      String name = StringUtils.popFirstNonOption(args);
+      final boolean printStats = StringUtils.popOption("-stats", args);
+      if (!args.isEmpty()) {
+        System.err.print("Can't understand arguments: " +
+          Joiner.on(" ").join(args) + "\n");
+        System.err.println("Usage is " + getShortUsage());
+        return 1;
+      }
+      DistributedFileSystem dfs = getDFS(conf);
+      TableListing.Builder builder = new TableListing.Builder().
+          addField("NAME", Justification.LEFT).
+          addField("OWNER", Justification.LEFT).
+          addField("GROUP", Justification.LEFT).
+          addField("MODE", Justification.LEFT).
+          addField("LIMIT", Justification.RIGHT).
+          addField("MAXTTL", Justification.RIGHT);
+      if (printStats) {
+        builder.
+            addField("BYTES_NEEDED", Justification.RIGHT).
+            addField("BYTES_CACHED", Justification.RIGHT).
+            addField("BYTES_OVERLIMIT", Justification.RIGHT).
+            addField("FILES_NEEDED", Justification.RIGHT).
+            addField("FILES_CACHED", Justification.RIGHT);
+      }
+      TableListing listing = builder.build();
+      int numResults = 0;
+      try {
+        RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
+        while (iter.hasNext()) {
+          CachePoolEntry entry = iter.next();
+          CachePoolInfo info = entry.getInfo();
+          LinkedList<String> row = new LinkedList<String>();
+          if (name == null || info.getPoolName().equals(name)) {
+            row.add(info.getPoolName());
+            row.add(info.getOwnerName());
+            row.add(info.getGroupName());
+            row.add(info.getMode() != null ? info.getMode().toString() : null);
+            Long limit = info.getLimit();
+            String limitString;
+            if (limit != null && limit.equals(CachePoolInfo.LIMIT_UNLIMITED)) {
+              limitString = "unlimited";
+            } else {
+              limitString = "" + limit;
+            }
+            row.add(limitString);
+            Long maxTtl = info.getMaxRelativeExpiryMs();
+            String maxTtlString = null;
+
+            if (maxTtl != null) {
+              if (maxTtl.longValue() == CachePoolInfo.RELATIVE_EXPIRY_NEVER) {
+                maxTtlString  = "never";
+              } else {
+                maxTtlString = DFSUtil.durationToString(maxTtl);
+              }
+            }
+            row.add(maxTtlString);
+            if (printStats) {
+              CachePoolStats stats = entry.getStats();
+              row.add(Long.toString(stats.getBytesNeeded()));
+              row.add(Long.toString(stats.getBytesCached()));
+              row.add(Long.toString(stats.getBytesOverlimit()));
+              row.add(Long.toString(stats.getFilesNeeded()));
+              row.add(Long.toString(stats.getFilesCached()));
+            }
+            listing.addRow(row.toArray(new String[] {}));
+            ++numResults;
+            if (name != null) {
+              break;
+            }
+          }
+        }
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+      System.out.print(String.format("Found %d result%s.\n", numResults,
+          (numResults == 1 ? "" : "s")));
+      if (numResults > 0) { 
+        System.out.print(listing);
+      }
+      // If there are no results, we return 1 (failure exit code);
+      // otherwise we return 0 (success exit code).
+      return (numResults == 0) ? 1 : 0;
+    }
+  }
+
+  private static class HelpCommand implements Command {
+    @Override
+    public String getName() {
+      return "-help";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[-help <command-name>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = getOptionDescriptionListing();
+      listing.addRow("<command-name>", "The command for which to get " +
+          "detailed help. If no command is specified, print detailed help for " +
+          "all commands");
+      return getShortUsage() + "\n" +
+        "Get detailed help about a command.\n\n" +
+        listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      if (args.size() == 0) {
+        for (Command command : COMMANDS) {
+          System.err.println(command.getLongUsage());
+        }
+        return 0;
+      }
+      if (args.size() != 1) {
+        System.out.println("You must give exactly one argument to -help.");
+        return 0;
+      }
+      String commandName = args.get(0);
+      // prepend a dash to match against the command names
+      Command command = determineCommand("-"+commandName);
+      if (command == null) {
+        System.err.print("Sorry, I don't know the command '" +
+          commandName + "'.\n");
+        System.err.print("Valid help command names are:\n");
+        String separator = "";
+        for (Command c : COMMANDS) {
+          System.err.print(separator + c.getName().substring(1));
+          separator = ", ";
+        }
+        System.err.print("\n");
+        return 1;
+      }
+      System.err.print(command.getLongUsage());
+      return 0;
+    }
+  }
+
+  private static Command[] COMMANDS = {
+    new AddCacheDirectiveInfoCommand(),
+    new ModifyCacheDirectiveInfoCommand(),
+    new ListCacheDirectiveInfoCommand(),
+    new RemoveCacheDirectiveInfoCommand(),
+    new RemoveCacheDirectiveInfosCommand(),
+    new AddCachePoolCommand(),
+    new ModifyCachePoolCommand(),
+    new RemoveCachePoolCommand(),
+    new ListCachePoolsCommand(),
+    new HelpCommand(),
+  };
+
+  private static void printUsage(boolean longUsage) {
+    System.err.println(
+        "Usage: bin/hdfs cacheadmin [COMMAND]");
+    for (Command command : COMMANDS) {
+      if (longUsage) {
+        System.err.print(command.getLongUsage());
+      } else {
+        System.err.print("          " + command.getShortUsage());
+      }
+    }
+    System.err.println();
+  }
+
+  private static Command determineCommand(String commandName) {
+    for (int i = 0; i < COMMANDS.length; i++) {
+      if (COMMANDS[i].getName().equals(commandName)) {
+        return COMMANDS[i];
+      }
+    }
+    return null;
+  }
+}

+ 284 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/TableListing.java

@@ -0,0 +1,284 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.WordUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * This class implements a "table listing" with column headers.
+ * 
+ * Example:
+ * 
+ * NAME   OWNER   GROUP   MODE       WEIGHT
+ * pool1  andrew  andrew  rwxr-xr-x     100
+ * pool2  andrew  andrew  rwxr-xr-x     100
+ * pool3  andrew  andrew  rwxr-xr-x     100
+ * 
+ */
+@InterfaceAudience.Private
+public class TableListing {
+  public enum Justification {
+    LEFT,
+    RIGHT;
+  }
+
+  private static class Column {
+    private final ArrayList<String> rows;
+    private final Justification justification;
+    private final boolean wrap;
+
+    private int wrapWidth = Integer.MAX_VALUE;
+    private int maxWidth;
+
+    Column(String title, Justification justification, boolean wrap) {
+      this.rows = new ArrayList<String>();
+      this.justification = justification;
+      this.wrap = wrap;
+      this.maxWidth = 0;
+      addRow(title);
+    }
+
+    private void addRow(String val) {
+      if (val == null) {
+        val = "";
+      }
+      if ((val.length() + 1) > maxWidth) {
+        maxWidth = val.length() + 1;
+      }
+      // Ceiling at wrapWidth, because it'll get wrapped
+      if (maxWidth > wrapWidth) {
+        maxWidth = wrapWidth;
+      }
+      rows.add(val);
+    }
+
+    private int getMaxWidth() {
+      return maxWidth;
+    }
+
+    private void setWrapWidth(int width) {
+      wrapWidth = width;
+      // Ceiling the maxLength at wrapWidth
+      if (maxWidth > wrapWidth) {
+        maxWidth = wrapWidth;
+      }
+      // Else we need to traverse through and find the real maxWidth
+      else {
+        maxWidth = 0;
+        for (int i=0; i<rows.size(); i++) {
+          int length = rows.get(i).length();
+          if (length > maxWidth) {
+            maxWidth = length;
+          }
+        }
+      }
+    }
+
+    /**
+     * Return the ith row of the column as a set of wrapped strings, each at
+     * most wrapWidth in length.
+     */
+    String[] getRow(int idx) {
+      String raw = rows.get(idx);
+      // Line-wrap if it's too long
+      String[] lines = new String[] {raw};
+      if (wrap) {
+        lines = WordUtils.wrap(lines[0], wrapWidth, "\n", true).split("\n");
+      }
+      for (int i=0; i<lines.length; i++) {
+        if (justification == Justification.LEFT) {
+          lines[i] = StringUtils.rightPad(lines[i], maxWidth);
+        } else if (justification == Justification.RIGHT) {
+          lines[i] = StringUtils.leftPad(lines[i], maxWidth);
+        }
+      }
+      return lines;
+    }
+  }
+
+  public static class Builder {
+    private final LinkedList<Column> columns = new LinkedList<Column>();
+    private boolean showHeader = true;
+    private int wrapWidth = Integer.MAX_VALUE;
+
+    /**
+     * Create a new Builder.
+     */
+    public Builder() {
+    }
+
+    public Builder addField(String title) {
+      return addField(title, Justification.LEFT, false);
+    }
+
+    public Builder addField(String title, Justification justification) {
+      return addField(title, justification, false);
+    }
+
+    public Builder addField(String title, boolean wrap) {
+      return addField(title, Justification.LEFT, wrap);
+    }
+
+    /**
+     * Add a new field to the Table under construction.
+     * 
+     * @param title Field title.
+     * @param justification Right or left justification. Defaults to left.
+     * @param wrap Width at which to auto-wrap the content of the cell.
+     *        Defaults to Integer.MAX_VALUE.
+     * @return This Builder object
+     */
+    public Builder addField(String title, Justification justification, 
+        boolean wrap) {
+      columns.add(new Column(title, justification, wrap));
+      return this;
+    }
+
+    /**
+     * Whether to hide column headers in table output
+     */
+    public Builder hideHeaders() {
+      this.showHeader = false;
+      return this;
+    }
+
+    /**
+     * Whether to show column headers in table output. This is the default.
+     */
+    public Builder showHeaders() {
+      this.showHeader = true;
+      return this;
+    }
+
+    /**
+     * Set the maximum width of a row in the TableListing. Must have one or
+     * more wrappable fields for this to take effect.
+     */
+    public Builder wrapWidth(int width) {
+      this.wrapWidth = width;
+      return this;
+    }
+
+    /**
+     * Create a new TableListing.
+     */
+    public TableListing build() {
+      return new TableListing(columns.toArray(new Column[0]), showHeader,
+          wrapWidth);
+    }
+  }
+
+  private final Column columns[];
+
+  private int numRows;
+  private boolean showHeader;
+  private int wrapWidth;
+
+  TableListing(Column columns[], boolean showHeader, int wrapWidth) {
+    this.columns = columns;
+    this.numRows = 0;
+    this.showHeader = showHeader;
+    this.wrapWidth = wrapWidth;
+  }
+
+  /**
+   * Add a new row.
+   *
+   * @param row    The row of objects to add-- one per column.
+   */
+  public void addRow(String... row) {
+    if (row.length != columns.length) {
+      throw new RuntimeException("trying to add a row with " + row.length +
+            " columns, but we have " + columns.length + " columns.");
+    }
+    for (int i = 0; i < columns.length; i++) {
+      columns[i].addRow(row[i]);
+    }
+    numRows++;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    // Calculate the widths of each column based on their maxWidths and
+    // the wrapWidth for the entire table
+    int width = (columns.length-1)*2; // inter-column padding
+    for (int i=0; i<columns.length; i++) {
+      width += columns[i].maxWidth;
+    }
+    // Decrease the column size of wrappable columns until the goal width
+    // is reached, or we can't decrease anymore
+    while (width > wrapWidth) {
+      boolean modified = false;
+      for (int i=0; i<columns.length; i++) {
+        Column column = columns[i];
+        if (column.wrap) {
+          int maxWidth = column.getMaxWidth();
+          if (maxWidth > 4) {
+            column.setWrapWidth(maxWidth-1);
+            modified = true;
+            width -= 1;
+            if (width <= wrapWidth) {
+              break;
+            }
+          }
+        }
+      }
+      if (!modified) {
+        break;
+      }
+    }
+
+    int startrow = 0;
+    if (!showHeader) {
+      startrow = 1;
+    }
+    String[][] columnLines = new String[columns.length][];
+    for (int i = startrow; i < numRows + 1; i++) {
+      int maxColumnLines = 0;
+      for (int j = 0; j < columns.length; j++) {
+        columnLines[j] = columns[j].getRow(i);
+        if (columnLines[j].length > maxColumnLines) {
+          maxColumnLines = columnLines[j].length;
+        }
+      }
+
+      for (int c = 0; c < maxColumnLines; c++) {
+        // First column gets no left-padding
+        String prefix = "";
+        for (int j = 0; j < columns.length; j++) {
+          // Prepend padding
+          builder.append(prefix);
+          prefix = " ";
+          if (columnLines[j].length > c) {
+            builder.append(columnLines[j][c]);
+          } else {
+            builder.append(StringUtils.repeat(" ", columns[j].maxWidth));
+          }
+        }
+        builder.append("\n");
+      }
+    }
+    return builder.toString();
+  }
+}

+ 39 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -127,7 +127,7 @@ class ImageLoaderCurrent implements ImageLoader {
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49, -50 };
+      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49, -50, -51 };
   private int imageVersion = 0;
   private int imageVersion = 0;
   
   
   private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
   private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
@@ -220,6 +220,9 @@ class ImageLoaderCurrent implements ImageLoader {
         processDelegationTokens(in, v);
         processDelegationTokens(in, v);
       }
       }
       
       
+      if (LayoutVersion.supports(Feature.CACHING, imageVersion)) {
+        processCacheManagerState(in, v);
+      }
       v.leaveEnclosingElement(); // FSImage
       v.leaveEnclosingElement(); // FSImage
       done = true;
       done = true;
     } finally {
     } finally {
@@ -231,6 +234,25 @@ class ImageLoaderCurrent implements ImageLoader {
     }
     }
   }
   }
 
 
+  /**
+   * Process CacheManager state from the fsimage.
+   */
+  private void processCacheManagerState(DataInputStream in, ImageVisitor v)
+      throws IOException {
+    v.visit(ImageElement.CACHE_NEXT_ENTRY_ID, in.readLong());
+    final int numPools = in.readInt();
+    for (int i=0; i<numPools; i++) {
+      v.visit(ImageElement.CACHE_POOL_NAME, Text.readString(in));
+      processCachePoolPermission(in, v);
+      v.visit(ImageElement.CACHE_POOL_WEIGHT, in.readInt());
+    }
+    final int numEntries = in.readInt();
+    for (int i=0; i<numEntries; i++) {
+      v.visit(ImageElement.CACHE_ENTRY_PATH, Text.readString(in));
+      v.visit(ImageElement.CACHE_ENTRY_REPLICATION, in.readShort());
+      v.visit(ImageElement.CACHE_ENTRY_POOL_NAME, Text.readString(in));
+    }
+  }
   /**
   /**
    * Process the Delegation Token related section in fsimage.
    * Process the Delegation Token related section in fsimage.
    * 
    * 
@@ -388,6 +410,22 @@ class ImageLoaderCurrent implements ImageLoader {
     v.leaveEnclosingElement(); // Permissions
     v.leaveEnclosingElement(); // Permissions
   }
   }
 
 
+  /**
+   * Extract CachePool permissions stored in the fsimage file.
+   *
+   * @param in Datastream to process
+   * @param v Visitor to walk over inodes
+   */
+  private void processCachePoolPermission(DataInputStream in, ImageVisitor v)
+      throws IOException {
+    v.visitEnclosingElement(ImageElement.PERMISSIONS);
+    v.visit(ImageElement.CACHE_POOL_OWNER_NAME, Text.readString(in));
+    v.visit(ImageElement.CACHE_POOL_GROUP_NAME, Text.readString(in));
+    FsPermission fsp = new FsPermission(in.readShort());
+    v.visit(ImageElement.CACHE_POOL_PERMISSION_STRING, fsp.toString());
+    v.leaveEnclosingElement(); // Permissions
+  }
+
   /**
   /**
    * Process the INode records stored in the fsimage.
    * Process the INode records stored in the fsimage.
    *
    *

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java

@@ -117,7 +117,19 @@ abstract class ImageVisitor {
     SNAPSHOT_DST_SNAPSHOT_ID,
     SNAPSHOT_DST_SNAPSHOT_ID,
     SNAPSHOT_LAST_SNAPSHOT_ID,
     SNAPSHOT_LAST_SNAPSHOT_ID,
     SNAPSHOT_REF_INODE_ID,
     SNAPSHOT_REF_INODE_ID,
-    SNAPSHOT_REF_INODE
+    SNAPSHOT_REF_INODE,
+
+    CACHE_NEXT_ENTRY_ID,
+    CACHE_NUM_POOLS,
+    CACHE_POOL_NAME,
+    CACHE_POOL_OWNER_NAME,
+    CACHE_POOL_GROUP_NAME,
+    CACHE_POOL_PERMISSION_STRING,
+    CACHE_POOL_WEIGHT,
+    CACHE_NUM_ENTRIES,
+    CACHE_ENTRY_PATH,
+    CACHE_ENTRY_REPLICATION,
+    CACHE_ENTRY_POOL_NAME
   }
   }
   
   
   /**
   /**

+ 16 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java

@@ -252,9 +252,24 @@ public class XMLUtils {
      * @return            the entry
      * @return            the entry
      */
      */
     public String getValue(String name) throws InvalidXmlException {
     public String getValue(String name) throws InvalidXmlException {
-      if (!subtrees.containsKey(name)) {
+      String ret = getValueOrNull(name);
+      if (ret == null) {
         throw new InvalidXmlException("no entry found for " + name);
         throw new InvalidXmlException("no entry found for " + name);
       }
       }
+      return ret;
+    }
+
+    /** 
+     * Pull a string entry from a stanza, or null.
+     *
+     * @param name        entry to look for
+     * 
+     * @return            the entry, or null if it was not found.
+     */
+    public String getValueOrNull(String name) throws InvalidXmlException {
+      if (!subtrees.containsKey(name)) {
+        return null;
+      }
       LinkedList <Stanza> l = subtrees.get(name);
       LinkedList <Stanza> l = subtrees.get(name);
       if (l.size() != 1) {
       if (l.size() != 1) {
         throw new InvalidXmlException("More than one value found for " + name);
         throw new InvalidXmlException("More than one value found for " + name);

+ 42 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -293,6 +293,8 @@ public class JsonUtil {
     m.put("dfsUsed", datanodeinfo.getDfsUsed());
     m.put("dfsUsed", datanodeinfo.getDfsUsed());
     m.put("remaining", datanodeinfo.getRemaining());
     m.put("remaining", datanodeinfo.getRemaining());
     m.put("blockPoolUsed", datanodeinfo.getBlockPoolUsed());
     m.put("blockPoolUsed", datanodeinfo.getBlockPoolUsed());
+    m.put("cacheCapacity", datanodeinfo.getCacheCapacity());
+    m.put("cacheUsed", datanodeinfo.getCacheUsed());
     m.put("lastUpdate", datanodeinfo.getLastUpdate());
     m.put("lastUpdate", datanodeinfo.getLastUpdate());
     m.put("xceiverCount", datanodeinfo.getXceiverCount());
     m.put("xceiverCount", datanodeinfo.getXceiverCount());
     m.put("networkLocation", datanodeinfo.getNetworkLocation());
     m.put("networkLocation", datanodeinfo.getNetworkLocation());
@@ -300,17 +302,37 @@ public class JsonUtil {
     return m;
     return m;
   }
   }
 
 
+  private static int getInt(Map<?, ?> m, String key, final int defaultValue) {
+    Object value = m.get(key);
+    if (value == null) {
+      return defaultValue;
+    }
+    return (int) (long) (Long) value;
+  }
+
+  private static long getLong(Map<?, ?> m, String key, final long defaultValue) {
+    Object value = m.get(key);
+    if (value == null) {
+      return defaultValue;
+    }
+    return (long) (Long) value;
+  }
+
+  private static String getString(Map<?, ?> m, String key,
+      final String defaultValue) {
+    Object value = m.get(key);
+    if (value == null) {
+      return defaultValue;
+    }
+    return (String) value;
+  }
+
   /** Convert a Json map to an DatanodeInfo object. */
   /** Convert a Json map to an DatanodeInfo object. */
   static DatanodeInfo toDatanodeInfo(final Map<?, ?> m)
   static DatanodeInfo toDatanodeInfo(final Map<?, ?> m)
       throws IOException {
       throws IOException {
     if (m == null) {
     if (m == null) {
       return null;
       return null;
     }
     }
-    
-    Object infoSecurePort = m.get("infoSecurePort");
-    if (infoSecurePort == null) {
-      infoSecurePort = 0l; // same as the default value in hdfs.proto
-    }
 
 
     // ipAddr and xferPort are the critical fields for accessing data.
     // ipAddr and xferPort are the critical fields for accessing data.
     // If any one of the two is missing, an exception needs to be thrown.
     // If any one of the two is missing, an exception needs to be thrown.
@@ -353,17 +375,19 @@ public class JsonUtil {
         (String)m.get("storageID"),
         (String)m.get("storageID"),
         xferPort,
         xferPort,
         (int)(long)(Long)m.get("infoPort"),
         (int)(long)(Long)m.get("infoPort"),
-        (int)(long)(Long)infoSecurePort,
+        getInt(m, "infoSecurePort", 0),
         (int)(long)(Long)m.get("ipcPort"),
         (int)(long)(Long)m.get("ipcPort"),
 
 
-        (Long)m.get("capacity"),
-        (Long)m.get("dfsUsed"),
-        (Long)m.get("remaining"),
-        (Long)m.get("blockPoolUsed"),
-        (Long)m.get("lastUpdate"),
-        (int)(long)(Long)m.get("xceiverCount"),
-        (String)m.get("networkLocation"),
-        AdminStates.valueOf((String)m.get("adminState")));
+        getLong(m, "capacity", 0l),
+        getLong(m, "dfsUsed", 0l),
+        getLong(m, "remaining", 0l),
+        getLong(m, "blockPoolUsed", 0l),
+        getLong(m, "cacheCapacity", 0l),
+        getLong(m, "cacheUsed", 0l),
+        getLong(m, "lastUpdate", 0l),
+        getInt(m, "xceiverCount", 0),
+        getString(m, "networkLocation", ""),
+        AdminStates.valueOf(getString(m, "adminState", "NORMAL")));
   }
   }
 
 
   /** Convert a DatanodeInfo[] to a Json array. */
   /** Convert a DatanodeInfo[] to a Json array. */
@@ -410,6 +434,7 @@ public class JsonUtil {
     m.put("startOffset", locatedblock.getStartOffset());
     m.put("startOffset", locatedblock.getStartOffset());
     m.put("block", toJsonMap(locatedblock.getBlock()));
     m.put("block", toJsonMap(locatedblock.getBlock()));
     m.put("locations", toJsonArray(locatedblock.getLocations()));
     m.put("locations", toJsonArray(locatedblock.getLocations()));
+    m.put("cachedLocations", toJsonArray(locatedblock.getCachedLocations()));
     return m;
     return m;
   }
   }
 
 
@@ -424,9 +449,11 @@ public class JsonUtil {
         (Object[])m.get("locations"));
         (Object[])m.get("locations"));
     final long startOffset = (Long)m.get("startOffset");
     final long startOffset = (Long)m.get("startOffset");
     final boolean isCorrupt = (Boolean)m.get("isCorrupt");
     final boolean isCorrupt = (Boolean)m.get("isCorrupt");
+    final DatanodeInfo[] cachedLocations = toDatanodeInfoArray(
+        (Object[])m.get("cachedLocations"));
 
 
     final LocatedBlock locatedblock = new LocatedBlock(b, locations,
     final LocatedBlock locatedblock = new LocatedBlock(b, locations,
-        null, null, startOffset, isCorrupt);
+        null, null, startOffset, isCorrupt, cachedLocations);
     locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
     locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
     return locatedblock;
     return locatedblock;
   }
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c

@@ -2375,7 +2375,7 @@ static int translateZCRException(JNIEnv *env, jthrowable exc)
         ret = EPROTONOSUPPORT;
         ret = EPROTONOSUPPORT;
         goto done;
         goto done;
     }
     }
-    ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+    ret = printExceptionAndFree(env, exc, PRINT_EXC_ALL,
             "hadoopZeroCopyRead: ZeroCopyCursor#read failed");
             "hadoopZeroCopyRead: ZeroCopyCursor#read failed");
 done:
 done:
     free(className);
     free(className);

+ 132 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -364,6 +364,122 @@ message IsFileClosedResponseProto {
   required bool result = 1;
   required bool result = 1;
 }
 }
 
 
+message CacheDirectiveInfoProto {
+  optional int64 id = 1;
+  optional string path = 2;
+  optional uint32 replication = 3;
+  optional string pool = 4;
+  optional CacheDirectiveInfoExpirationProto expiration = 5;
+}
+
+message CacheDirectiveInfoExpirationProto {
+  required int64 millis = 1;
+  required bool isRelative = 2;
+}
+
+message CacheDirectiveStatsProto {
+  required int64 bytesNeeded = 1;
+  required int64 bytesCached = 2;
+  required int64 filesNeeded = 3;
+  required int64 filesCached = 4;
+  required bool hasExpired = 5;
+}
+
+enum CacheFlagProto {
+  FORCE = 0x01;    // Ignore pool resource limits
+}
+
+message AddCacheDirectiveRequestProto {
+  required CacheDirectiveInfoProto info = 1;
+  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
+}
+
+message AddCacheDirectiveResponseProto {
+  required int64 id = 1;
+}
+
+message ModifyCacheDirectiveRequestProto {
+  required CacheDirectiveInfoProto info = 1;
+  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
+}
+
+message ModifyCacheDirectiveResponseProto {
+}
+
+message RemoveCacheDirectiveRequestProto {
+  required int64 id = 1;
+}
+
+message RemoveCacheDirectiveResponseProto {
+}
+
+message ListCacheDirectivesRequestProto {
+  required int64 prevId = 1;
+  required CacheDirectiveInfoProto filter = 2;
+}
+
+message CacheDirectiveEntryProto {
+  required CacheDirectiveInfoProto info = 1;
+  required CacheDirectiveStatsProto stats = 2;
+}
+
+message ListCacheDirectivesResponseProto {
+  repeated CacheDirectiveEntryProto elements = 1;
+  required bool hasMore = 2;
+}
+
+message CachePoolInfoProto {
+  optional string poolName = 1;
+  optional string ownerName = 2;
+  optional string groupName = 3;
+  optional int32 mode = 4;
+  optional int64 limit = 5;
+  optional int64 maxRelativeExpiry = 6;
+}
+
+message CachePoolStatsProto {
+  required int64 bytesNeeded = 1;
+  required int64 bytesCached = 2;
+  required int64 bytesOverlimit = 3;
+  required int64 filesNeeded = 4;
+  required int64 filesCached = 5;
+}
+
+message AddCachePoolRequestProto {
+  required CachePoolInfoProto info = 1;
+}
+
+message AddCachePoolResponseProto { // void response
+}
+
+message ModifyCachePoolRequestProto {
+  required CachePoolInfoProto info = 1;
+}
+
+message ModifyCachePoolResponseProto { // void response
+}
+
+message RemoveCachePoolRequestProto {
+  required string poolName = 1;
+}
+
+message RemoveCachePoolResponseProto { // void response
+}
+
+message ListCachePoolsRequestProto {
+  required string prevPoolName = 1;
+}
+
+message ListCachePoolsResponseProto {
+  repeated CachePoolEntryProto entries = 1;
+  required bool hasMore = 2;
+}
+
+message CachePoolEntryProto {
+  required CachePoolInfoProto info = 1;
+  required CachePoolStatsProto stats = 2;
+}
+
 message GetFileLinkInfoRequestProto {
 message GetFileLinkInfoRequestProto {
   required string src = 1;
   required string src = 1;
 }
 }
@@ -546,6 +662,22 @@ service ClientNamenodeProtocol {
       returns(ListCorruptFileBlocksResponseProto);
       returns(ListCorruptFileBlocksResponseProto);
   rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);
   rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);
   rpc getFileInfo(GetFileInfoRequestProto) returns(GetFileInfoResponseProto);
   rpc getFileInfo(GetFileInfoRequestProto) returns(GetFileInfoResponseProto);
+  rpc addCacheDirective(AddCacheDirectiveRequestProto)
+      returns (AddCacheDirectiveResponseProto);
+  rpc modifyCacheDirective(ModifyCacheDirectiveRequestProto)
+      returns (ModifyCacheDirectiveResponseProto);
+  rpc removeCacheDirective(RemoveCacheDirectiveRequestProto)
+      returns (RemoveCacheDirectiveResponseProto);
+  rpc listCacheDirectives(ListCacheDirectivesRequestProto)
+      returns (ListCacheDirectivesResponseProto);
+  rpc addCachePool(AddCachePoolRequestProto)
+      returns(AddCachePoolResponseProto);
+  rpc modifyCachePool(ModifyCachePoolRequestProto)
+      returns(ModifyCachePoolResponseProto);
+  rpc removeCachePool(RemoveCachePoolRequestProto)
+      returns(RemoveCachePoolResponseProto);
+  rpc listCachePools(ListCachePoolsRequestProto)
+      returns(ListCachePoolsResponseProto);
   rpc getFileLinkInfo(GetFileLinkInfoRequestProto)
   rpc getFileLinkInfo(GetFileLinkInfoRequestProto)
       returns(GetFileLinkInfoResponseProto);
       returns(GetFileLinkInfoResponseProto);
   rpc getContentSummary(GetContentSummaryRequestProto)
   rpc getContentSummary(GetContentSummaryRequestProto)

+ 44 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto

@@ -70,6 +70,7 @@ message DatanodeCommandProto {
     RegisterCommand = 5;
     RegisterCommand = 5;
     UnusedUpgradeCommand = 6;
     UnusedUpgradeCommand = 6;
     NullDatanodeCommand = 7;
     NullDatanodeCommand = 7;
+    BlockIdCommand = 8;
   }
   }
 
 
   required Type cmdType = 1;    // Type of the command
   required Type cmdType = 1;    // Type of the command
@@ -82,6 +83,7 @@ message DatanodeCommandProto {
   optional FinalizeCommandProto finalizeCmd = 5;
   optional FinalizeCommandProto finalizeCmd = 5;
   optional KeyUpdateCommandProto keyUpdateCmd = 6;
   optional KeyUpdateCommandProto keyUpdateCmd = 6;
   optional RegisterCommandProto registerCmd = 7;
   optional RegisterCommandProto registerCmd = 7;
+  optional BlockIdCommandProto blkIdCmd = 8;
 }
 }
 
 
 /**
 /**
@@ -102,7 +104,7 @@ message BlockCommandProto {
   enum Action {  
   enum Action {  
     TRANSFER = 1;   // Transfer blocks to another datanode
     TRANSFER = 1;   // Transfer blocks to another datanode
     INVALIDATE = 2; // Invalidate blocks
     INVALIDATE = 2; // Invalidate blocks
-    SHUTDOWN = 3; // Shutdown the datanode
+    SHUTDOWN = 3;   // Shutdown the datanode
   }
   }
 
 
   required Action action = 1;
   required Action action = 1;
@@ -112,6 +114,20 @@ message BlockCommandProto {
   repeated StorageUuidsProto targetStorageUuids = 5;
   repeated StorageUuidsProto targetStorageUuids = 5;
 }
 }
 
 
+/**
+ * Command to instruct datanodes to perform certain action
+ * on the given set of block IDs.
+ */
+message BlockIdCommandProto {
+  enum Action {
+    CACHE = 1;
+    UNCACHE = 2;
+  }
+  required Action action = 1;
+  required string blockPoolId = 2;
+  repeated uint64 blockIds = 3 [packed=true];
+}
+
 /**
 /**
  * List of blocks to be recovered by the datanode
  * List of blocks to be recovered by the datanode
  */
  */
@@ -165,6 +181,8 @@ message RegisterDatanodeResponseProto {
  * xmitsInProgress - number of transfers from this datanode to others
  * xmitsInProgress - number of transfers from this datanode to others
  * xceiverCount - number of active transceiver threads
  * xceiverCount - number of active transceiver threads
  * failedVolumes - number of failed volumes
  * failedVolumes - number of failed volumes
+ * cacheCapacity - total cache capacity available at the datanode
+ * cacheUsed - amount of cache used
  */
  */
 message HeartbeatRequestProto {
 message HeartbeatRequestProto {
   required DatanodeRegistrationProto registration = 1; // Datanode info
   required DatanodeRegistrationProto registration = 1; // Datanode info
@@ -172,6 +190,8 @@ message HeartbeatRequestProto {
   optional uint32 xmitsInProgress = 3 [ default = 0 ];
   optional uint32 xmitsInProgress = 3 [ default = 0 ];
   optional uint32 xceiverCount = 4 [ default = 0 ];
   optional uint32 xceiverCount = 4 [ default = 0 ];
   optional uint32 failedVolumes = 5 [ default = 0 ];
   optional uint32 failedVolumes = 5 [ default = 0 ];
+  optional uint64 cacheCapacity = 6 [ default = 0 ];
+  optional uint64 cacheUsed = 7 [default = 0 ];
 }
 }
 
 
 message StorageReportProto {
 message StorageReportProto {
@@ -209,9 +229,11 @@ message HeartbeatResponseProto {
 /**
 /**
  * registration - datanode registration information
  * registration - datanode registration information
  * blockPoolID  - block pool ID of the reported blocks
  * blockPoolID  - block pool ID of the reported blocks
- * blocks       - each block is represented as two longs in the array.
+ * blocks       - each block is represented as multiple longs in the array.
  *                first long represents block ID
  *                first long represents block ID
  *                second long represents length
  *                second long represents length
+ *                third long represents gen stamp
+ *                fourth long (if under construction) represents replica state
  */
  */
 message BlockReportRequestProto {
 message BlockReportRequestProto {
   required DatanodeRegistrationProto registration = 1;
   required DatanodeRegistrationProto registration = 1;
@@ -234,6 +256,21 @@ message BlockReportResponseProto {
   optional DatanodeCommandProto cmd = 1;
   optional DatanodeCommandProto cmd = 1;
 } 
 } 
 
 
+/**
+ * registration - datanode registration information
+ * blockPoolId  - block pool ID of the reported blocks
+ * blocks       - representation of blocks as longs for efficiency reasons
+ */
+message CacheReportRequestProto {
+  required DatanodeRegistrationProto registration = 1;
+  required string blockPoolId = 2;
+  repeated uint64 blocks = 3 [packed=true];
+}
+
+message CacheReportResponseProto {
+  optional DatanodeCommandProto cmd = 1;
+}
+
 /**
 /**
  * Data structure to send received or deleted block information
  * Data structure to send received or deleted block information
  * from datanode to namenode.
  * from datanode to namenode.
@@ -351,6 +388,11 @@ service DatanodeProtocolService {
    */
    */
   rpc blockReport(BlockReportRequestProto) returns(BlockReportResponseProto);
   rpc blockReport(BlockReportRequestProto) returns(BlockReportResponseProto);
 
 
+  /**
+   * Report cached blocks at a datanode to the namenode
+   */
+  rpc cacheReport(CacheReportRequestProto) returns(CacheReportResponseProto);
+
   /**
   /**
    * Incremental block report from the DN. This contains info about recently
    * Incremental block report from the DN. This contains info about recently
    * received and deleted blocks, as well as when blocks start being
    * received and deleted blocks, as well as when blocks start being

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

@@ -86,6 +86,8 @@ message DatanodeInfoProto {
   }
   }
 
 
   optional AdminState adminState = 10 [default = NORMAL];
   optional AdminState adminState = 10 [default = NORMAL];
+  optional uint64 cacheCapacity = 11 [default = 0];
+  optional uint64 cacheUsed = 12 [default = 0];
 }
 }
 
 
 /**
 /**
@@ -144,6 +146,7 @@ message LocatedBlockProto {
                                         // their locations are not part of this object
                                         // their locations are not part of this object
 
 
   required hadoop.common.TokenProto blockToken = 5;
   required hadoop.common.TokenProto blockToken = 5;
+  repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
   repeated StorageTypeProto storageTypes = 7;
   repeated StorageTypeProto storageTypes = 7;
   repeated string storageIDs = 8;
   repeated string storageIDs = 8;
 }
 }

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

@@ -1475,6 +1475,102 @@
   </description>
   </description>
 </property>
 </property>
 
 
+<property>
+  <name>dfs.namenode.path.based.cache.block.map.allocation.percent</name>
+  <value>0.25</value>
+  <description>
+    The percentage of the Java heap which we will allocate to the cached blocks
+    map.  The cached blocks map is a hash map which uses chained hashing.
+    Smaller maps may be accessed more slowly if the number of cached blocks is
+    large; larger maps will consume more memory.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.max.locked.memory</name>
+  <value>0</value>
+  <description>
+    The amount of memory in bytes to use for caching of block replicas in
+    memory on the datanode. The datanode's maximum locked memory soft ulimit
+    (RLIMIT_MEMLOCK) must be set to at least this value, else the datanode
+    will abort on startup.
+
+    By default, this parameter is set to 0, which disables in-memory caching.
+
+    If the native libraries are not available to the DataNode, this
+    configuration has no effect.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.list.cache.directives.num.responses</name>
+  <value>100</value>
+  <description>
+    This value controls the number of cache directives that the NameNode will
+    send over the wire in response to a listDirectives RPC.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.list.cache.pools.num.responses</name>
+  <value>100</value>
+  <description>
+    This value controls the number of cache pools that the NameNode will
+    send over the wire in response to a listPools RPC.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.path.based.cache.refresh.interval.ms</name>
+  <value>300000</value>
+  <description>
+    The amount of milliseconds between subsequent path cache rescans.  Path
+    cache rescans are when we calculate which blocks should be cached, and on
+    what datanodes.
+
+    By default, this parameter is set to 300000, which is five minutes.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.path.based.cache.retry.interval.ms</name>
+  <value>60000</value>
+  <description>
+    When the NameNode needs to uncache something that is cached, or cache
+    something that is not cached, it must direct the DataNodes to do so by
+    sending a DNA_CACHE or DNA_UNCACHE command in response to a DataNode
+    heartbeat.  This parameter controls how frequently the NameNode will
+    resend these commands.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.fsdatasetcache.max.threads.per.volume</name>
+  <value>4</value>
+  <description>
+    The maximum number of threads per volume to use for caching new data
+    on the datanode. These threads consume both I/O and CPU. This can affect
+    normal datanode operations.
+  </description>
+</property>
+
+<property>
+  <name>dfs.cachereport.intervalMsec</name>
+  <value>10000</value>
+  <description>
+    Determines cache reporting interval in milliseconds.  After this amount of
+    time, the DataNode sends a full report of its cache state to the NameNode.
+    The NameNode uses the cache report to update its map of cached blocks to
+    DataNode locations.
+
+    This configuration has no effect if in-memory caching has been disabled by
+    setting dfs.datanode.max.locked.memory to 0 (which is the default).
+
+    If the native libraries are not available to the DataNode, this
+    configuration has no effect.
+  </description>
+</property>
+
 <property>
 <property>
   <name>dfs.namenode.edit.log.autoroll.multiplier.threshold</name>
   <name>dfs.namenode.edit.log.autoroll.multiplier.threshold</name>
   <value>2.0</value>
   <value>2.0</value>

+ 301 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/CentralizedCacheManagement.apt.vm

@@ -0,0 +1,301 @@
+~~ Licensed 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. See accompanying LICENSE file.
+
+  ---
+  Hadoop Distributed File System-${project.version} - Centralized Cache Management in HDFS
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Centralized Cache Management in HDFS
+
+  \[ {{{./index.html}Go Back}} \]
+
+%{toc|section=1|fromDepth=2|toDepth=4}
+
+* {Background}
+
+  Normally, HDFS relies on the operating system to cache data it reads from disk.
+  However, HDFS can also be configured to use centralized cache management. Under
+  centralized cache management, the HDFS NameNode itself decides which blocks
+  should be cached, and where they should be cached.
+
+  Centralized cache management has several advantages. First of all, it
+  prevents frequently used block files from being evicted from memory. This is
+  particularly important when the size of the working set exceeds the size of
+  main memory, which is true for many big data applications. Secondly, when
+  HDFS decides what should be cached, it can let clients know about this
+  information through the getFileBlockLocations API. Finally, when the DataNode
+  knows a block is locked into memory, it can provide access to that block via
+  mmap.
+
+* {Use Cases}
+
+  Centralized cache management is most useful for files which are accessed very
+  often. For example, a "fact table" in Hive which is often used in joins is a
+  good candidate for caching. On the other hand, when running a classic
+  "word count" MapReduce job which counts the number of words in each
+  document, there may not be any good candidates for caching, since all the
+  files may be accessed exactly once.
+
+* {Architecture}
+
+[images/caching.png] Caching Architecture
+
+  With centralized cache management, the NameNode coordinates all caching
+  across the cluster. It receives cache information from each DataNode via the
+  cache report, a periodic message that describes all the blocks IDs cached on
+  a given DataNode. The NameNode will reply to DataNode heartbeat messages
+  with commands telling it which blocks to cache and which to uncache.
+
+  The NameNode stores a set of path cache directives, which tell it which files
+  to cache. The NameNode also stores a set of cache pools, which are groups of
+  cache directives.  These directives and pools are persisted to the edit log
+  and fsimage, and will be loaded if the cluster is restarted.
+
+  Periodically, the NameNode rescans the namespace, to see which blocks need to
+  be cached based on the current set of path cache directives. Rescans are also
+  triggered by relevant user actions, such as adding or removing a cache
+  directive or removing a cache pool.
+
+  Cache directives also may specific a numeric cache replication, which is the
+  number of replicas to cache.  This number may be equal to or smaller than the
+  file's block replication.  If multiple cache directives cover the same file
+  with different cache replication settings, then the highest cache replication
+  setting is applied.
+
+  We do not currently cache blocks which are under construction, corrupt, or
+  otherwise incomplete.  If a cache directive covers a symlink, the symlink
+  target is not cached.
+
+  Caching is currently done on a per-file basis, although we would like to add
+  block-level granularity in the future.
+
+* {Interface}
+
+  The NameNode stores a list of "cache directives."  These directives contain a
+  path as well as the number of times blocks in that path should be replicated.
+
+  Paths can be either directories or files. If the path specifies a file, that
+  file is cached. If the path specifies a directory, all the files in the
+  directory will be cached. However, this process is not recursive-- only the
+  direct children of the directory will be cached.
+
+** {hdfs cacheadmin Shell}
+
+  Path cache directives can be created by the <<<hdfs cacheadmin
+  -addDirective>>> command and removed via the <<<hdfs cacheadmin
+  -removeDirective>>> command. To list the current path cache directives, use
+  <<<hdfs cacheadmin -listDirectives>>>. Each path cache directive has a
+  unique 64-bit ID number which will not be reused if it is deleted.  To remove
+  all path cache directives with a specified path, use <<<hdfs cacheadmin
+  -removeDirectives>>>.
+
+  Directives are grouped into "cache pools."  Each cache pool gets a share of
+  the cluster's resources. Additionally, cache pools are used for
+  authentication. Cache pools have a mode, user, and group, similar to regular
+  files. The same authentication rules are applied as for normal files. So, for
+  example, if the mode is 0777, any user can add or remove directives from the
+  cache pool. If the mode is 0644, only the owner can write to the cache pool,
+  but anyone can read from it. And so forth.
+
+  Cache pools are identified by name. They can be created by the <<<hdfs
+  cacheAdmin -addPool>>> command, modified by the <<<hdfs cacheadmin
+  -modifyPool>>> command, and removed via the <<<hdfs cacheadmin
+  -removePool>>> command. To list the current cache pools, use <<<hdfs
+  cacheAdmin -listPools>>>
+
+*** {addDirective}
+
+  Usage: <<<hdfs cacheadmin -addDirective -path <path> -replication <replication> -pool <pool-name> >>>
+
+  Add a new cache directive.
+
+*--+--+
+\<path\> | A path to cache. The path can be a directory or a file.
+*--+--+
+\<replication\> | The cache replication factor to use. Defaults to 1.
+*--+--+
+\<pool-name\> | The pool to which the directive will be added. You must have write permission on the cache pool in order to add new directives.
+*--+--+
+
+*** {removeDirective}
+
+  Usage: <<<hdfs cacheadmin -removeDirective <id> >>>
+
+  Remove a cache directive.
+
+*--+--+
+\<id\> | The id of the cache directive to remove.  You must have write permission on the pool of the directive in order to remove it.  To see a list of cachedirective IDs, use the -listDirectives command.
+*--+--+
+
+*** {removeDirectives}
+
+  Usage: <<<hdfs cacheadmin -removeDirectives <path> >>>
+
+  Remove every cache directive with the specified path.
+
+*--+--+
+\<path\> | The path of the cache directives to remove.  You must have write permission on the pool of the directive in order to remove it.  To see a list of cache directives, use the -listDirectives command.
+*--+--+
+
+*** {listDirectives}
+
+  Usage: <<<hdfs cacheadmin -listDirectives [-path <path>] [-pool <pool>] >>>
+
+  List cache directives.
+
+*--+--+
+\<path\> | List only cache directives with this path. Note that if there is a cache directive for <path> in a cache pool that we don't have read access for, it will not be listed.
+*--+--+
+\<pool\> | List only path cache directives in that pool.
+*--+--+
+
+*** {addPool}
+
+  Usage: <<<hdfs cacheadmin -addPool <name> [-owner <owner>] [-group <group>] [-mode <mode>] [-weight <weight>] >>>
+
+  Add a new cache pool.
+
+*--+--+
+\<name\> | Name of the new pool.
+*--+--+
+\<owner\> | Username of the owner of the pool. Defaults to the current user.
+*--+--+
+\<group\> | Group of the pool. Defaults to the primary group name of the current user.
+*--+--+
+\<mode\> | UNIX-style permissions for the pool. Permissions are specified in octal, e.g. 0755. By default, this is set to 0755.
+*--+--+
+\<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 100.
+*--+--+
+
+*** {modifyPool}
+
+  Usage: <<<hdfs cacheadmin -modifyPool <name> [-owner <owner>] [-group <group>] [-mode <mode>] [-weight <weight>] >>>
+
+  Modifies the metadata of an existing cache pool.
+
+*--+--+
+\<name\> | Name of the pool to modify.
+*--+--+
+\<owner\> | Username of the owner of the pool.
+*--+--+
+\<group\> | Groupname of the group of the pool.
+*--+--+
+\<mode\> | Unix-style permissions of the pool in octal.
+*--+--+
+\<weight\> | Weight of the pool.
+*--+--+
+
+*** {removePool}
+
+  Usage: <<<hdfs cacheadmin -removePool <name> >>>
+
+  Remove a cache pool. This also uncaches paths associated with the pool.
+
+*--+--+
+\<name\> | Name of the cache pool to remove.
+*--+--+
+
+*** {listPools}
+
+  Usage: <<<hdfs cacheadmin -listPools [name] >>>
+
+  Display information about one or more cache pools, e.g. name, owner, group,
+  permissions, etc.
+
+*--+--+
+\<name\> | If specified, list only the named cache pool.
+*--+--+
+
+*** {help}
+
+  Usage: <<<hdfs cacheadmin -help <command-name> >>>
+
+  Get detailed help about a command.
+
+*--+--+
+\<command-name\> | The command for which to get detailed help. If no command is specified, print detailed help for all commands.
+*--+--+
+
+* {Configuration}
+
+** {Native Libraries}
+
+  In order to lock block files into memory, the DataNode relies on native JNI
+  code found in <<<libhadoop.so>>>. Be sure to
+  {{{../hadoop-common/NativeLibraries.html}enable JNI}} if you are using HDFS
+  centralized cache management.
+
+** {Configuration Properties}
+
+*** Required
+
+  Be sure to configure the following:
+
+  * dfs.datanode.max.locked.memory
+
+    The DataNode will treat this as the maximum amount of memory it can use for
+    its cache. When setting this value, please remember that you will need space
+    in memory for other things, such as the Java virtual machine (JVM) itself
+    and the operating system's page cache.
+
+*** Optional
+
+  The following properties are not required, but may be specified for tuning:
+
+  * dfs.namenode.path.based.cache.refresh.interval.ms
+
+    The NameNode will use this as the amount of milliseconds between subsequent
+    path cache rescans.  This calculates the blocks to cache and each DataNode
+    containing a replica of the block that should cache it.
+
+    By default, this parameter is set to 300000, which is five minutes.
+
+  * dfs.datanode.fsdatasetcache.max.threads.per.volume
+
+    The DataNode will use this as the maximum number of threads per volume to
+    use for caching new data.
+
+    By default, this parameter is set to 4.
+
+  * dfs.cachereport.intervalMsec
+
+    The DataNode will use this as the amount of milliseconds between sending a
+    full report of its cache state to the NameNode.
+
+    By default, this parameter is set to 10000, which is 10 seconds.
+
+  * dfs.namenode.path.based.cache.block.map.allocation.percent
+
+    The percentage of the Java heap which we will allocate to the cached blocks
+    map.  The cached blocks map is a hash map which uses chained hashing.
+    Smaller maps may be accessed more slowly if the number of cached blocks is
+    large; larger maps will consume more memory.  The default is 0.25 percent.
+
+** {OS Limits}
+
+  If you get the error "Cannot start datanode because the configured max
+  locked memory size... is more than the datanode's available RLIMIT_MEMLOCK
+  ulimit," that means that the operating system is imposing a lower limit
+  on the amount of memory that you can lock than what you have configured. To
+  fix this, you must adjust the ulimit -l value that the DataNode runs with.
+  Usually, this value is configured in <<</etc/security/limits.conf>>>.
+  However, it will vary depending on what operating system and distribution
+  you are using.
+
+  You will know that you have correctly configured this value when you can run
+  <<<ulimit -l>>> from the shell and get back either a higher value than what
+  you have configured with <<<dfs.datanode.max.locked.memory>>>, or the string
+  "unlimited," indicating that there is no limit.  Note that it's typical for
+  <<<ulimit -l>>> to output the memory lock limit in KB, but
+  dfs.datanode.max.locked.memory must be specified in bytes.

BIN
hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/caching.png


+ 141 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCacheAdminCLI.java

@@ -0,0 +1,141 @@
+/**
+ * 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.cli;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.cli.util.CLICommand;
+import org.apache.hadoop.cli.util.CLICommandCacheAdmin;
+import org.apache.hadoop.cli.util.CLICommandTypes;
+import org.apache.hadoop.cli.util.CLITestCmd;
+import org.apache.hadoop.cli.util.CacheAdminCmdExecutor;
+import org.apache.hadoop.cli.util.CommandExecutor;
+import org.apache.hadoop.cli.util.CommandExecutor.Result;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HDFSPolicyProvider;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.tools.CacheAdmin;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.xml.sax.SAXException;
+
+public class TestCacheAdminCLI extends CLITestHelper {
+
+  public static final Log LOG = LogFactory.getLog(TestCacheAdminCLI.class);
+
+  protected MiniDFSCluster dfsCluster = null;
+  protected FileSystem fs = null;
+  protected String namenode = null;
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    conf.setClass(PolicyProvider.POLICY_PROVIDER_CONFIG,
+        HDFSPolicyProvider.class, PolicyProvider.class);
+
+    // Many of the tests expect a replication value of 1 in the output
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
+
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+
+    dfsCluster.waitClusterUp();
+    namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");
+    username = System.getProperty("user.name");
+
+    fs = dfsCluster.getFileSystem();
+    assertTrue("Not a HDFS: "+fs.getUri(),
+               fs instanceof DistributedFileSystem);
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+    Thread.sleep(2000);
+    super.tearDown();
+  }
+
+  @Override
+  protected String getTestFile() {
+    return "testCacheAdminConf.xml";
+  }
+
+  @Override
+  protected TestConfigFileParser getConfigParser() {
+    return new TestConfigFileParserCacheAdmin();
+  }
+
+  private class TestConfigFileParserCacheAdmin extends
+      CLITestHelper.TestConfigFileParser {
+    @Override
+    public void endElement(String uri, String localName, String qName)
+        throws SAXException {
+      if (qName.equals("cache-admin-command")) {
+        if (testCommands != null) {
+          testCommands.add(new CLITestCmdCacheAdmin(charString,
+              new CLICommandCacheAdmin()));
+        } else if (cleanupCommands != null) {
+          cleanupCommands.add(new CLITestCmdCacheAdmin(charString,
+              new CLICommandCacheAdmin()));
+        }
+      } else {
+        super.endElement(uri, localName, qName);
+      }
+    }
+  }
+
+  private class CLITestCmdCacheAdmin extends CLITestCmd {
+
+    public CLITestCmdCacheAdmin(String str, CLICommandTypes type) {
+      super(str, type);
+    }
+
+    @Override
+    public CommandExecutor getExecutor(String tag)
+        throws IllegalArgumentException {
+      if (getType() instanceof CLICommandCacheAdmin) {
+        return new CacheAdminCmdExecutor(tag, new CacheAdmin(conf));
+      }
+      return super.getExecutor(tag);
+    }
+  }
+
+  @Override
+  protected Result execute(CLICommand cmd) throws Exception {
+    return cmd.getExecutor("").executeCommand(cmd.getCmd());
+  }
+
+  @Test
+  @Override
+  public void testAll () {
+    super.testAll();
+  }
+}

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandCacheAdmin.java

@@ -0,0 +1,21 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.cli.util;
+
+public class CLICommandCacheAdmin implements CLICommandTypes {
+}

Some files were not shown because too many files changed in this diff