1
0
فهرست منبع

HDFS-5096. Automatically cache new data added to a cached path (contributed by Colin Patrick McCabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4949@1532924 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 11 سال پیش
والد
کامیت
3cc7a38a53
29فایلهای تغییر یافته به همراه1949 افزوده شده و 2156 حذف شده
  1. 373 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IntrusiveCollection.java
  2. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-4949.txt
  3. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  4. 0 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  5. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  6. 170 36
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  7. 0 607
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationManager.java
  8. 361 210
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  9. 0 129
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationPolicy.java
  10. 70 121
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  11. 57 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  12. 41 34
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
  13. 0 67
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateStoredBlocks.java
  14. 11 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
  15. 0 271
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReportProcessor.java
  16. 0 44
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UncacheBlocks.java
  17. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  18. 0 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java
  19. 331 178
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  20. 14 52
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  21. 3 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  22. 18 53
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  23. 0 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  24. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  25. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  26. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  27. 0 256
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCacheReplicationManager.java
  28. 151 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCachedBlocksList.java
  29. 319 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java

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

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-4949.txt

@@ -66,6 +66,9 @@ HDFS-4949 (Unreleased)
     HDFS-5359. Allow LightWeightGSet#Iterator to remove elements.
     (Contributed by Colin Patrick McCabe)
 
+    HDFS-5096. Automatically cache new data added to a cached path.
+    (Contributed by Colin Patrick McCabe)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 

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

@@ -205,6 +205,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES =
       "dfs.namenode.list.cache.descriptors.num.responses";
   public static final int     DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_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
   public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";

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

@@ -60,17 +60,6 @@ public interface BlockCollection {
    */
   public short getBlockReplication();
 
-  /**
-   * Set cache replication factor for the collection
-   */
-  public void setCacheReplication(short cacheReplication);
-
-  /**
-   * Get cache replication factor for the collection
-   * @return cache replication value
-   */
-  public short getCacheReplication();
-
   /**
    * Get the name of the collection.
    */

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

@@ -85,7 +85,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     this.bc = bc;
   }
 
-  DatanodeDescriptor getDatanode(int index) {
+  public DatanodeDescriptor getDatanode(int index) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
     return (DatanodeDescriptor)triplets[index*3];
@@ -153,7 +153,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     return info;
   }
 
-  int getCapacity() {
+  public int getCapacity() {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert triplets.length % 3 == 0 : "Malformed BlockInfo";
     return triplets.length / 3;

+ 170 - 36
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -77,13 +77,14 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
 
 /**
  * Keeps information related to the blocks stored in the Hadoop cluster.
  */
 @InterfaceAudience.Private
-public class BlockManager extends ReportProcessor {
+public class BlockManager {
 
   static final Log LOG = LogFactory.getLog(BlockManager.class);
   public static final Log blockLog = NameNode.blockStateChangeLog;
@@ -162,7 +163,7 @@ public class BlockManager extends ReportProcessor {
   final CorruptReplicasMap corruptReplicas = new CorruptReplicasMap();
 
   /** Blocks to be invalidated. */
-  private final InvalidateStoredBlocks invalidateBlocks;
+  private final InvalidateBlocks invalidateBlocks;
   
   /**
    * After a failover, over-replicated blocks may not be handled
@@ -218,6 +219,7 @@ public class BlockManager extends ReportProcessor {
   final boolean encryptDataTransfer;
   
   // Max number of blocks to log info about during a block report.
+  private final long maxNumBlocksToLog;
 
   /**
    * When running inside a Standby node, the node may receive block reports
@@ -235,11 +237,10 @@ public class BlockManager extends ReportProcessor {
   
   public BlockManager(final Namesystem namesystem, final FSClusterStats stats,
       final Configuration conf) throws IOException {
-    super(conf);
     this.namesystem = namesystem;
     datanodeManager = new DatanodeManager(this, namesystem, conf);
     heartbeatManager = datanodeManager.getHeartbeatManager();
-    invalidateBlocks = new InvalidateStoredBlocks(datanodeManager);
+    invalidateBlocks = new InvalidateBlocks(datanodeManager);
 
     // Compute the map capacity by allocating 2% of total memory
     blocksMap = new BlocksMap(DEFAULT_MAP_LOAD_FACTOR);
@@ -299,7 +300,11 @@ public class BlockManager extends ReportProcessor {
     this.encryptDataTransfer =
         conf.getBoolean(DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY,
             DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEFAULT);
-
+    
+    this.maxNumBlocksToLog =
+        conf.getLong(DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
+            DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT);
+    
     LOG.info("defaultReplication         = " + defaultReplication);
     LOG.info("maxReplication             = " + maxReplication);
     LOG.info("minReplication             = " + minReplication);
@@ -999,7 +1004,6 @@ public class BlockManager extends ReportProcessor {
    * Adds block to list of blocks which will be invalidated on specified
    * datanode and log the operation
    */
-  @Override  // ReportProcessor
   void addToInvalidates(final Block block, final DatanodeInfo datanode) {
     invalidateBlocks.add(block, datanode, true);
   }
@@ -1045,8 +1049,7 @@ public class BlockManager extends ReportProcessor {
     markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason), dn);
   }
 
-  @Override // ReportProcessor
-  void markBlockAsCorrupt(BlockToMarkCorrupt b,
+  private void markBlockAsCorrupt(BlockToMarkCorrupt b,
                                   DatanodeInfo dn) throws IOException {
     DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
     if (node == null) {
@@ -1056,7 +1059,7 @@ public class BlockManager extends ReportProcessor {
 
     BlockCollection bc = b.corrupted.getBlockCollection();
     if (bc == null) {
-      blockLogInfo("#markBlockAsCorrupt: " + b
+      blockLog.info("BLOCK markBlockAsCorrupt: " + b
           + " cannot be marked as corrupt as it does not belong to any file");
       addToInvalidates(b.corrupted, node);
       return;
@@ -1120,9 +1123,6 @@ public class BlockManager extends ReportProcessor {
     this.shouldPostponeBlocksFromFuture  = postpone;
   }
 
-  public boolean shouldPostponeBlocksFromFuture() {
-    return this.shouldPostponeBlocksFromFuture;
-  }
 
   private void postponeBlock(Block blk) {
     if (postponedMisreplicatedBlocks.add(blk)) {
@@ -1544,6 +1544,61 @@ public class BlockManager extends ReportProcessor {
        */
     }
   }
+  
+  /**
+   * StatefulBlockInfo is used to build the "toUC" list, which is a list of
+   * updates to the information about under-construction blocks.
+   * Besides the block in question, it provides the ReplicaState
+   * reported by the datanode in the block report. 
+   */
+  private static class StatefulBlockInfo {
+    final BlockInfoUnderConstruction storedBlock;
+    final ReplicaState reportedState;
+    
+    StatefulBlockInfo(BlockInfoUnderConstruction storedBlock, 
+        ReplicaState reportedState) {
+      this.storedBlock = storedBlock;
+      this.reportedState = reportedState;
+    }
+  }
+  
+  /**
+   * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
+   * list of blocks that should be considered corrupt due to a block report.
+   */
+  private static class BlockToMarkCorrupt {
+    /** The corrupted block in a datanode. */
+    final BlockInfo corrupted;
+    /** The corresponding block stored in the BlockManager. */
+    final BlockInfo stored;
+    /** The reason to mark corrupt. */
+    final String reason;
+    
+    BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason) {
+      Preconditions.checkNotNull(corrupted, "corrupted is null");
+      Preconditions.checkNotNull(stored, "stored is null");
+
+      this.corrupted = corrupted;
+      this.stored = stored;
+      this.reason = reason;
+    }
+
+    BlockToMarkCorrupt(BlockInfo stored, String reason) {
+      this(stored, stored, reason);
+    }
+
+    BlockToMarkCorrupt(BlockInfo stored, long gs, String reason) {
+      this(new BlockInfo(stored), stored, reason);
+      //the corrupted block in datanode has a different generation stamp
+      corrupted.setGenerationStamp(gs);
+    }
+
+    @Override
+    public String toString() {
+      return corrupted + "("
+          + (corrupted == stored? "same as stored": "stored=" + stored) + ")";
+    }
+  }
 
   /**
    * The given datanode is reporting all its blocks.
@@ -1635,6 +1690,46 @@ public class BlockManager extends ReportProcessor {
     }
   }
   
+  private void processReport(final DatanodeDescriptor node,
+      final BlockListAsLongs report) throws IOException {
+    // Normal case:
+    // Modify the (block-->datanode) map, according to the difference
+    // between the old and new block report.
+    //
+    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
+    Collection<Block> toRemove = new LinkedList<Block>();
+    Collection<Block> toInvalidate = new LinkedList<Block>();
+    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
+    Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
+    reportDiff(node, report, toAdd, toRemove, toInvalidate, toCorrupt, toUC);
+
+    // Process the blocks on each queue
+    for (StatefulBlockInfo b : toUC) { 
+      addStoredBlockUnderConstruction(b.storedBlock, node, b.reportedState);
+    }
+    for (Block b : toRemove) {
+      removeStoredBlock(b, node);
+    }
+    int numBlocksLogged = 0;
+    for (BlockInfo b : toAdd) {
+      addStoredBlock(b, node, null, numBlocksLogged < maxNumBlocksToLog);
+      numBlocksLogged++;
+    }
+    if (numBlocksLogged > maxNumBlocksToLog) {
+      blockLog.info("BLOCK* processReport: logged info for " + maxNumBlocksToLog
+          + " of " + numBlocksLogged + " reported.");
+    }
+    for (Block b : toInvalidate) {
+      blockLog.info("BLOCK* processReport: "
+          + b + " on " + node + " size " + b.getNumBytes()
+          + " does not belong to any file");
+      addToInvalidates(b, node);
+    }
+    for (BlockToMarkCorrupt b : toCorrupt) {
+      markBlockAsCorrupt(b, node);
+    }
+  }
+
   /**
    * processFirstBlockReport is intended only for processing "initial" block
    * reports, the first block report received from a DN after it registers.
@@ -1697,6 +1792,44 @@ public class BlockManager extends ReportProcessor {
     }
   }
 
+  private void reportDiff(DatanodeDescriptor dn, 
+      BlockListAsLongs newReport, 
+      Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
+      Collection<Block> toRemove,           // remove from DatanodeDescriptor
+      Collection<Block> toInvalidate,       // should be removed from DN
+      Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
+      Collection<StatefulBlockInfo> toUC) { // add to under-construction list
+    // place a delimiter in the list which separates blocks 
+    // that have been reported from those that have not
+    BlockInfo delimiter = new BlockInfo(new Block(), 1);
+    boolean added = dn.addBlock(delimiter);
+    assert added : "Delimiting block cannot be present in the node";
+    int headIndex = 0; //currently the delimiter is in the head of the list
+    int curIndex;
+
+    if (newReport == null)
+      newReport = new BlockListAsLongs();
+    // scan the report and process newly reported blocks
+    BlockReportIterator itBR = newReport.getBlockReportIterator();
+    while(itBR.hasNext()) {
+      Block iblk = itBR.next();
+      ReplicaState iState = itBR.getCurrentReplicaState();
+      BlockInfo storedBlock = processReportedBlock(dn, iblk, iState,
+                                  toAdd, toInvalidate, toCorrupt, toUC);
+      // move block to the head of the list
+      if (storedBlock != null && (curIndex = storedBlock.findDatanode(dn)) >= 0) {
+        headIndex = dn.moveBlockToHead(storedBlock, curIndex, headIndex);
+      }
+    }
+    // collect blocks that have not been reported
+    // all of them are next to the delimiter
+    Iterator<? extends Block> it = new DatanodeDescriptor.BlockIterator(
+        delimiter.getNext(0), dn);
+    while(it.hasNext())
+      toRemove.add(it.next());
+    dn.removeBlock(delimiter);
+  }
+
   /**
    * Process a block replica reported by the data-node.
    * No side effects except adding to the passed-in Collections.
@@ -1728,8 +1861,7 @@ public class BlockManager extends ReportProcessor {
    * @return the up-to-date stored block, if it should be kept.
    *         Otherwise, null.
    */
-  @Override // ReportProcessor
-  BlockInfo processReportedBlock(final DatanodeDescriptor dn, 
+  private BlockInfo processReportedBlock(final DatanodeDescriptor dn, 
       final Block block, final ReplicaState reportedState, 
       final Collection<BlockInfo> toAdd, 
       final Collection<Block> toInvalidate, 
@@ -1956,7 +2088,6 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     }
   }
   
-  @Override // ReportProcessor
   void addStoredBlockUnderConstruction(
       BlockInfoUnderConstruction block, 
       DatanodeDescriptor node, 
@@ -2012,8 +2143,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    * needed replications if this takes care of the problem.
    * @return the block that is stored in blockMap.
    */
-  @Override // ReportProcessor
-  Block addStoredBlock(final BlockInfo block,
+  private Block addStoredBlock(final BlockInfo block,
                                DatanodeDescriptor node,
                                DatanodeDescriptor delNodeHint,
                                boolean logEveryBlock)
@@ -2028,7 +2158,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     }
     if (storedBlock == null || storedBlock.getBlockCollection() == null) {
       // If this block does not belong to anyfile, then we are done.
-      blockLogInfo("#addStoredBlock: " + block + " on "
+      blockLog.info("BLOCK* addStoredBlock: " + block + " on "
           + node + " size " + block.getNumBytes()
           + " but it does not belong to any file");
       // we could add this block to invalidate set of this datanode.
@@ -2050,7 +2180,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       }
     } else {
       curReplicaDelta = 0;
-      blockLogWarn("#addStoredBlock: "
+      blockLog.warn("BLOCK* addStoredBlock: "
           + "Redundant addStoredBlock request received for " + storedBlock
           + " on " + node + " size " + storedBlock.getNumBytes());
     }
@@ -2108,6 +2238,20 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     return storedBlock;
   }
 
+  private void logAddStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) {
+    if (!blockLog.isInfoEnabled()) {
+      return;
+    }
+    
+    StringBuilder sb = new StringBuilder(500);
+    sb.append("BLOCK* addStoredBlock: blockMap updated: ")
+      .append(node)
+      .append(" is added to ");
+    storedBlock.appendStringTo(sb);
+    sb.append(" size " )
+      .append(storedBlock.getNumBytes());
+    blockLog.info(sb);
+  }
   /**
    * Invalidate corrupt replicas.
    * <p>
@@ -2989,6 +3133,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         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 */
   public int numOfUnderReplicatedBlocks() {
     return neededReplications.size();
@@ -3129,21 +3280,4 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
   public void shutdown() {
     blocksMap.close();
   }
-
-  @Override // ReportProcessor
-  int moveBlockToHead(DatanodeDescriptor dn, BlockInfo storedBlock,
-      int curIndex, int headIndex) {
-    return dn.moveBlockToHead(storedBlock, curIndex, headIndex);
-  }
-
-  @Override // ReportProcessor
-  boolean addBlock(DatanodeDescriptor dn, BlockInfo block) {
-    return dn.addBlock(block);
-  }
-
-  @Override // ReportProcessor
-  boolean removeBlock(DatanodeDescriptor dn, BlockInfo block) {
-    return dn.removeBlock(block);
-  }
-
 }

+ 0 - 607
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationManager.java

@@ -1,607 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-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.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.Namesystem;
-import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
-import org.apache.hadoop.hdfs.util.LightWeightHashSet;
-import org.apache.hadoop.util.Time;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * Analogue of the BlockManager class for cached replicas. Maintains the mapping
- * of cached blocks to datanodes via processing datanode cache reports. Based on
- * these reports and addition and removal of caching directives in the
- * CacheManager, the CacheReplicationManager will schedule caching and uncaching
- * work.
- * 
- * The CacheReplicationManager does not have a separate lock, so depends on
- * taking the namesystem lock as appropriate.
- */
-@InterfaceAudience.LimitedPrivate({"HDFS"})
-public class CacheReplicationManager extends ReportProcessor {
-
-  private static final Log LOG =
-      LogFactory.getLog(CacheReplicationManager.class);
-
-  // Statistics
-  private volatile long pendingCacheBlocksCount = 0L;
-  private volatile long underCachedBlocksCount = 0L;
-  private volatile long scheduledCacheBlocksCount = 0L;
-
-  /** Used by metrics */
-  public long getPendingCacheBlocksCount() {
-    return pendingCacheBlocksCount;
-  }
-  /** Used by metrics */
-  public long getUnderCachedBlocksCount() {
-    return underCachedBlocksCount;
-  }
-  /** Used by metrics */
-  public long getScheduledCacheBlocksCount() {
-    return scheduledCacheBlocksCount;
-  }
-  /** Used by metrics */
-  public long getPendingBlocksToUncacheCount() {
-    return blocksToUncache.numBlocks();
-  }
-
-  private final Namesystem namesystem;
-  private final BlockManager blockManager;
-  private final DatanodeManager datanodeManager;
-  private final boolean isCachingEnabled;
-
-  /**
-   * Mapping of blocks to datanodes where the block is cached
-   */
-  final BlocksMap cachedBlocksMap;
-  /**
-   * Blocks to be uncached
-   */
-  private final UncacheBlocks blocksToUncache;
-  /**
-   * Blocks that need to be cached
-   */
-  private final LightWeightHashSet<Block> neededCacheBlocks;
-  /**
-   * Blocks that are being cached
-   */
-  private final PendingReplicationBlocks pendingCacheBlocks;
-
-  /**
-   * Executor for the CacheReplicationMonitor thread
-   */
-  private ExecutorService monitor = null;
-
-  private final Configuration conf;
-
-  public CacheReplicationManager(final Namesystem namesystem,
-      final BlockManager blockManager, final DatanodeManager datanodeManager,
-      final FSClusterStats stats, final Configuration conf) throws IOException {
-    super(conf);
-    this.namesystem = namesystem;
-    this.blockManager = blockManager;
-    this.datanodeManager = datanodeManager;
-    this.conf = conf;
-    isCachingEnabled = conf.getBoolean(
-        DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY,
-        DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_DEFAULT);
-    if (isCachingEnabled) {
-      cachedBlocksMap = new BlocksMap(BlockManager.DEFAULT_MAP_LOAD_FACTOR);
-      blocksToUncache = new UncacheBlocks();
-      pendingCacheBlocks = new PendingReplicationBlocks(1000 * conf.getInt(
-          DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
-          DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT));
-      neededCacheBlocks = new LightWeightHashSet<Block>();
-    } else {
-      cachedBlocksMap = null;
-      blocksToUncache = null;
-      pendingCacheBlocks = null;
-      neededCacheBlocks = null;
-    }
-  }
-
-  public void activate() {
-    if (isCachingEnabled) {
-      pendingCacheBlocks.start();
-      this.monitor = Executors.newSingleThreadExecutor(
-          new ThreadFactoryBuilder()
-          .setDaemon(true)
-          .setNameFormat(CacheReplicationMonitor.class.toString())
-          .build());
-      monitor.submit(new CacheReplicationMonitor(namesystem, blockManager,
-          datanodeManager, this, blocksToUncache, neededCacheBlocks,
-          pendingCacheBlocks, conf));
-      monitor.shutdown();
-    }
-  }
-
-  public void close() {
-    if (isCachingEnabled) {
-      monitor.shutdownNow();
-      try {
-        monitor.awaitTermination(3000, TimeUnit.MILLISECONDS);
-      } catch (InterruptedException e) {
-      }
-      pendingCacheBlocks.stop();
-      cachedBlocksMap.close();
-    }
-  }
-
-  public void clearQueues() {
-    if (isCachingEnabled) {
-      blocksToUncache.clear();
-      synchronized (neededCacheBlocks) {
-        neededCacheBlocks.clear();
-      }
-      pendingCacheBlocks.clear();
-    }
-  }
-
-  public boolean isCachingEnabled() {
-    return isCachingEnabled;
-  }
-
-  /**
-   * @return desired cache replication factor of the block
-   */
-  short getCacheReplication(Block block) {
-    final BlockCollection bc = blockManager.blocksMap.getBlockCollection(block);
-    return bc == null ? 0 : bc.getCacheReplication();
-  }
-
-  public void setCachedLocations(LocatedBlock block) {
-    BlockInfo blockInfo = cachedBlocksMap.getStoredBlock(
-        block.getBlock().getLocalBlock());
-    for (int i=0; i<blockInfo.numNodes(); i++) {
-      block.addCachedLoc(blockInfo.getDatanode(i));
-    }
-  }
-
-  /**
-   * Returns the number of cached replicas of a block
-   */
-  short getNumCached(Block block) {
-    Iterator<DatanodeDescriptor> it = cachedBlocksMap.nodeIterator(block);
-    short numCached = 0;
-    while (it.hasNext()) {
-      it.next();
-      numCached++;
-    }
-    return numCached;
-  }
-
-  /**
-   * The given datanode is reporting all of its cached blocks.
-   * Update the cache state of blocks in the block map.
-   */
-  public void processCacheReport(final DatanodeID nodeID, final String poolId,
-      final BlockListAsLongs newReport) throws IOException {
-    if (!isCachingEnabled) {
-      String error = "cacheReport received from datanode " + nodeID
-          + " but caching is disabled on the namenode ("
-          + DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY + ")";
-      LOG.warn(error + ", ignoring");
-      throw new IOException(error);
-    }
-    namesystem.writeLock();
-    final long startTime = Time.now(); //after acquiring write lock
-    final long endTime;
-    try {
-      final DatanodeDescriptor node = datanodeManager.getDatanode(nodeID);
-      if (node == null || !node.isAlive) {
-        throw new IOException(
-            "processCacheReport from dead or unregistered node: " + nodeID);
-      }
-
-      // TODO: do an optimized initial cache report while in startup safemode
-      if (namesystem.isInStartupSafeMode()) {
-        blockLogInfo("#processCacheReport: "
-            + "discarded cache report from " + nodeID
-            + " because namenode still in startup phase");
-        return;
-      }
-
-      processReport(node, newReport);
-
-      // TODO: process postponed blocks reported while a standby
-      //rescanPostponedMisreplicatedBlocks();
-    } finally {
-      endTime = Time.now();
-      namesystem.writeUnlock();
-    }
-
-    // Log the block report processing stats from Namenode perspective
-    final NameNodeMetrics metrics = NameNode.getNameNodeMetrics();
-    if (metrics != null) {
-      metrics.addCacheBlockReport((int) (endTime - startTime));
-    }
-    blockLogInfo("#processCacheReport: from "
-        + nodeID + ", blocks: " + newReport.getNumberOfBlocks()
-        + ", processing time: " + (endTime - startTime) + " msecs");
-  }
-
-  @Override // ReportProcessor
-  void markBlockAsCorrupt(BlockToMarkCorrupt b, DatanodeInfo dn)
-      throws IOException {
-    throw new UnsupportedOperationException("Corrupt blocks should not be in"
-        + " the cache report");
-  }
-
-  @Override // ReportProcessor
-  void addToInvalidates(final Block b, final DatanodeInfo node) {
-    blocksToUncache.add(b, node, true);
-  }
-
-  @Override // ReportProcessor
-  void addStoredBlockUnderConstruction(
-      BlockInfoUnderConstruction storedBlock, DatanodeDescriptor node,
-      ReplicaState reportedState) {
-    throw new UnsupportedOperationException("Under-construction blocks"
-        + " should not be in the cache report");
-  }
-
-  @Override // ReportProcessor
-  int moveBlockToHead(DatanodeDescriptor dn, BlockInfo storedBlock,
-      int curIndex, int headIndex) {
-    return dn.moveCachedBlockToHead(storedBlock, curIndex, headIndex);
-  }
-
-  @Override // ReportProcessor
-  boolean addBlock(DatanodeDescriptor dn, BlockInfo block) {
-    return dn.addCachedBlock(block);
-  }
-
-  @Override // ReportProcessor
-  boolean removeBlock(DatanodeDescriptor dn, BlockInfo block) {
-    return dn.removeCachedBlock(block);
-  }
-
-  /**
-   * Similar to processReportedBlock. Simpler since it doesn't need to worry
-   * about under construction and corrupt replicas.
-   * 
-   * @return Updated BlockInfo for the block if it should be kept, null if
-   * it is to be invalidated.
-   */
-  @Override // ReportProcessor
-  BlockInfo processReportedBlock(final DatanodeDescriptor dn,
-      final Block block, final ReplicaState reportedState,
-      final Collection<BlockInfo> toAdd,
-      final Collection<Block> toInvalidate,
-      Collection<BlockToMarkCorrupt> toCorrupt,
-      Collection<StatefulBlockInfo> toUC) {
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Reported cached block " + block
-          + " on " + dn + " size " + block.getNumBytes()
-          + " replicaState = " + reportedState);
-    }
-
-    final boolean shouldPostponeBlocksFromFuture =
-        blockManager.shouldPostponeBlocksFromFuture();
-    if (shouldPostponeBlocksFromFuture &&
-        namesystem.isGenStampInFuture(block)) {
-      // TODO: queuing cache operations on the standby
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("processReportedBlock: block " + block + " has a "
-            + "genstamp from the future and namenode is in standby mode,"
-            + " ignoring");
-      }
-      return null;
-    }
-
-    BlockInfo storedBlock = blockManager.blocksMap.getStoredBlock(block);
-    if (storedBlock == null) {
-      // If blocksMap does not contain reported block id,
-      // the BlockManager will take care of invalidating it, and the datanode
-      // will automatically uncache at that point.
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("processReportedBlock: block " + block + " not found "
-            + "in blocksMap, ignoring");
-      }
-      return null;
-    }
-
-    BlockUCState ucState = storedBlock.getBlockUCState();
-
-    // Datanodes currently only will cache completed replicas.
-    // Let's just invalidate anything that's not completed and the right
-    // genstamp and number of bytes.
-    if (!ucState.equals(BlockUCState.COMPLETE) ||
-        block.getGenerationStamp() != storedBlock.getGenerationStamp() ||
-        block.getNumBytes() != storedBlock.getNumBytes()) {
-      if (shouldPostponeBlocksFromFuture) {
-        // TODO: queuing cache operations on the standby
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("processReportedBlock: block " + block + " has a "
-              + "mismatching genstamp or length and namenode is in standby"
-              + " mode, ignoring");
-        }
-        return null;
-      } else {
-        toInvalidate.add(block);
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("processReportedBlock: block " + block + " scheduled"
-              + " for uncaching because it is misreplicated"
-              + " or under construction.");
-        }
-        return null;
-      }
-    }
-
-    // It's a keeper
-
-    // Could be present in blocksMap and not in cachedBlocksMap, add it
-    BlockInfo cachedBlock = cachedBlocksMap.getStoredBlock(block);
-    if (cachedBlock == null) {
-      cachedBlock = new BlockInfo(block, 0);
-      cachedBlocksMap.addBlockCollection(cachedBlock,
-          storedBlock.getBlockCollection());
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("In memory blockUCState = " + ucState);
-    }
-
-    // Ignore replicas that are already scheduled for removal
-    if (blocksToUncache.contains(dn.getStorageID(), block)) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("processReportedBlock: block " + block + " is already"
-            + " scheduled to be uncached, not adding it to the cachedBlocksMap");
-      }
-      return cachedBlock;
-    }
-
-    // add replica if not already present in the cached block map
-    if (reportedState == ReplicaState.FINALIZED
-        && cachedBlock.findDatanode(dn) < 0) {
-      toAdd.add(cachedBlock);
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("processReportedBlock: block " + block + " scheduled"
-          + " to be added to cachedBlocksMap");
-    }
-    return cachedBlock;
-  }
-
-  /**
-   * Modify (cached block-->datanode) map with a newly cached block. Remove
-   * block from set of needed cache replications if this takes care of the
-   * problem.
-   * 
-   * @return the block that is stored in cachedBlockMap.
-   */
-  @Override // ReportProcessor
-  Block addStoredBlock(final BlockInfo block, DatanodeDescriptor node,
-      DatanodeDescriptor delNodeHint, boolean logEveryBlock) throws IOException {
-    assert block != null && namesystem.hasWriteLock();
-    BlockInfo cachedBlock = block;
-    if (cachedBlock == null || cachedBlock.getBlockCollection() == null) {
-      // If this block does not belong to anyfile, then we are done.
-      blockLogInfo("#addStoredBlock: " + block + " on "
-          + node + " size " + block.getNumBytes()
-          + " but it does not belong to any file");
-      // we could add this block to invalidate set of this datanode.
-      // it will happen in next block report otherwise.
-      return block;
-    }
-
-    BlockCollection bc = cachedBlock.getBlockCollection();
-
-    // add block to the datanode
-    boolean added = node.addCachedBlock(cachedBlock);
-
-    int curReplicaDelta;
-    if (added) {
-      curReplicaDelta = 1;
-      if (logEveryBlock) {
-        logAddStoredBlock(cachedBlock, node);
-      }
-    } else {
-      curReplicaDelta = 0;
-      blockLogWarn("#addStoredBlock: "
-          + "Redundant addCachedBlock request received for " + cachedBlock
-          + " on " + node + " size " + cachedBlock.getNumBytes());
-    }
-
-    // Remove it from pending list if present
-    pendingCacheBlocks.decrement(block, node);
-
-    // Now check for completion of blocks and safe block count
-    int numCachedReplicas = getNumCached(cachedBlock);
-    int numEffectiveCachedReplica = numCachedReplicas
-      + pendingCacheBlocks.getNumReplicas(cachedBlock);
-
-    // if file is under construction, then done for now
-    if (bc instanceof MutableBlockCollection) {
-      return cachedBlock;
-    }
-
-    // do not try to handle over/under-replicated blocks during first safe mode
-    if (!namesystem.isPopulatingReplQueues()) {
-      return cachedBlock;
-    }
-
-    // Under-replicated
-    short cacheReplication = bc.getCacheReplication();
-    if (numEffectiveCachedReplica >= cacheReplication) {
-      synchronized (neededCacheBlocks) {
-        neededCacheBlocks.remove(cachedBlock);
-      }
-    } else {
-      updateNeededCaching(cachedBlock, curReplicaDelta, 0);
-    }
-
-    // Over-replicated, we don't need this new replica
-    if (numEffectiveCachedReplica > cacheReplication) {
-      blocksToUncache.add(cachedBlock, node, true);
-    }
-
-    return cachedBlock;
-  }
-
-  /**
-   * Modify (cached block-->datanode) map. Possibly generate replication tasks,
-   * if the removed block is still valid.
-   */
-  @Override // ReportProcessor
-  void removeStoredBlock(Block block, DatanodeDescriptor node) {
-    blockLogDebug("#removeStoredBlock: " + block + " from " + node);
-    assert (namesystem.hasWriteLock());
-    {
-      if (!cachedBlocksMap.removeNode(block, node)) {
-        blockLogDebug("#removeStoredBlock: "
-            + block + " has already been removed from node " + node);
-        return;
-      }
-
-      // Prune the block from the map if it's the last cache replica
-      if (cachedBlocksMap.getStoredBlock(block).numNodes() == 0) {
-        cachedBlocksMap.removeBlock(block);
-      }
-
-      //
-      // It's possible that the block was removed because of a datanode
-      // failure. If the block is still valid, check if replication is
-      // necessary. In that case, put block on a possibly-will-
-      // be-replicated list.
-      //
-      BlockCollection bc = blockManager.blocksMap.getBlockCollection(block);
-      if (bc != null) {
-        updateNeededCaching(block, -1, 0);
-      }
-    }
-  }
-
-  /**
-   * Reduce cache replication factor to the new replication by randomly
-   * choosing replicas to invalidate.
-   */
-  private void processOverCachedBlock(final Block block,
-      final short replication) {
-    assert namesystem.hasWriteLock();
-    List<DatanodeDescriptor> nodes = getSafeReplicas(cachedBlocksMap, block);
-    List<DatanodeDescriptor> targets =
-        CacheReplicationPolicy.chooseTargetsToUncache(nodes, replication);
-    for (DatanodeDescriptor dn: targets) {
-      blocksToUncache.add(block, dn, true);
-    }
-  }
-
-  /** Set replication for the blocks. */
-  public void setCacheReplication(final short oldRepl, final short newRepl,
-      final String src, final Block... blocks) {
-    if (!isCachingEnabled) {
-      LOG.warn("Attempted to set cache replication for " + src + " but caching"
-          + " is disabled (" + DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY
-          + "), ignoring");
-      return;
-    }
-    if (newRepl == oldRepl) {
-      return;
-    }
-
-    // update needReplication priority queues
-    for (Block b : blocks) {
-      updateNeededCaching(b, 0, newRepl-oldRepl);
-    }
-
-    if (oldRepl > newRepl) {
-      // old replication > the new one; need to remove copies
-      LOG.info("Decreasing cache replication from " + oldRepl + " to " + newRepl
-          + " for " + src);
-      for (Block b : blocks) {
-        processOverCachedBlock(b, newRepl);
-      }
-    } else { // replication factor is increased
-      LOG.info("Increasing cache replication from " + oldRepl + " to " + newRepl
-          + " for " + src);
-    }
-  }
-
-  /** updates a block in under replicated queue */
-  private void updateNeededCaching(final Block block,
-      final int curReplicasDelta, int expectedReplicasDelta) {
-    namesystem.writeLock();
-    try {
-      if (!namesystem.isPopulatingReplQueues()) {
-        return;
-      }
-      final int numCached = getNumCached(block);
-      final int curExpectedReplicas = getCacheReplication(block);
-      if (numCached < curExpectedReplicas) {
-        neededCacheBlocks.add(block);
-      } else {
-        synchronized (neededCacheBlocks) {
-          neededCacheBlocks.remove(block);
-        }
-      }
-    } finally {
-      namesystem.writeUnlock();
-    }
-  }
-
-  /**
-   * Return the safe replicas (not corrupt or decomissioning/decommissioned) of
-   * a block in a BlocksMap
-   */
-  List<DatanodeDescriptor> getSafeReplicas(BlocksMap map, Block block) {
-    List<DatanodeDescriptor> nodes = new ArrayList<DatanodeDescriptor>(3);
-    Collection<DatanodeDescriptor> corrupted =
-        blockManager.corruptReplicas.getNodes(block);
-    Iterator<DatanodeDescriptor> it = map.nodeIterator(block);
-    while (it.hasNext()) {
-      DatanodeDescriptor dn = it.next();
-      // Don't count a decommissioned or decommissioning nodes
-      if (dn.isDecommissioned() || dn.isDecommissionInProgress()) {
-        continue;
-      }
-      // Don't count a corrupted node
-      if (corrupted != null && corrupted.contains(dn)) {
-        continue;
-      }
-      nodes.add(dn);
-    }
-    return nodes;
-  }
-}

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

@@ -19,284 +19,435 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
-import java.util.ArrayList;
-import java.util.HashMap;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map.Entry;
+import java.util.Random;
 
 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.hdfs.DFSConfigKeys;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.Namesystem;
-import org.apache.hadoop.hdfs.util.LightWeightHashSet;
+import org.apache.hadoop.hdfs.protocol.PathBasedCacheEntry;
+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.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;
 
 /**
- * Periodically computes new replication work. This consists of two tasks:
- * 
- * 1) Assigning blocks in the neededCacheBlocks to datanodes where they will be
- * cached. This moves them to the pendingCacheBlocks list.
- * 
- * 2) Placing caching tasks in pendingCacheBlocks that have timed out
- * back into neededCacheBlocks for reassignment.
+ * 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"})
-class CacheReplicationMonitor implements Runnable {
+public class CacheReplicationMonitor extends Thread implements Closeable {
 
   private static final Log LOG =
       LogFactory.getLog(CacheReplicationMonitor.class);
 
-  private static final Log blockLog = NameNode.blockStateChangeLog;
+  private final FSNamesystem namesystem;
 
-  private final Namesystem namesystem;
   private final BlockManager blockManager;
-  private final DatanodeManager datanodeManager;
-  private final CacheReplicationManager cacheReplManager;
 
-  private final UncacheBlocks blocksToUncache;
-  private final LightWeightHashSet<Block> neededCacheBlocks;
-  private final PendingReplicationBlocks pendingCacheBlocks;
+  private final CacheManager cacheManager;
+
+  private final GSet<CachedBlock, CachedBlock> cachedBlocks;
 
   /**
-   * Re-check period for computing cache replication work
+   * Pseudorandom number source
    */
-  private final long cacheReplicationRecheckInterval;
+  private final Random random = new Random();
 
-  public CacheReplicationMonitor(Namesystem namesystem,
-      BlockManager blockManager, DatanodeManager datanodeManager,
-      CacheReplicationManager cacheReplManager,
-      UncacheBlocks blocksToUncache,
-      LightWeightHashSet<Block> neededCacheBlocks,
-      PendingReplicationBlocks pendingCacheBlocks,
-      Configuration conf) {
-    this.namesystem = namesystem;
-    this.blockManager = blockManager;
-    this.datanodeManager = datanodeManager;
-    this.cacheReplManager = cacheReplManager;
+  /**
+   * The interval at which we scan the namesystem for caching changes.
+   */
+  private final long intervalMs;
 
-    this.blocksToUncache = blocksToUncache;
-    this.neededCacheBlocks = neededCacheBlocks;
-    this.pendingCacheBlocks = pendingCacheBlocks;
+  /**
+   * True if we should rescan immediately, regardless of how much time
+   * elapsed since the previous scan.
+   */
+  private boolean rescanImmediately;
+
+  /**
+   * The monotonic time at which the current scan started.
+   */
+  private long scanTimeMs;
+
+  /**
+   * Mark status of the current scan.
+   */
+  private boolean mark = false;
 
-    this.cacheReplicationRecheckInterval = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 
-        DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000L;
+  /**
+   * True if this monitor should terminate.
+   */
+  private boolean shutdown;
+
+  /**
+   * 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) {
+    this.namesystem = namesystem;
+    this.blockManager = namesystem.getBlockManager();
+    this.cacheManager = cacheManager;
+    this.cachedBlocks = cacheManager.getCachedBlocks();
+    this.intervalMs = intervalMs;
   }
 
   @Override
   public void run() {
-    LOG.info("CacheReplicationMonitor is starting");
-    while (namesystem.isRunning()) {
-      try {
-        computeCachingWork();
-        processPendingCachingWork();
-        Thread.sleep(cacheReplicationRecheckInterval);
-      } catch (Throwable t) {
-        if (!namesystem.isRunning()) {
-          LOG.info("Stopping CacheReplicationMonitor.");
-          if (!(t instanceof InterruptedException)) {
-            LOG.info("CacheReplicationMonitor received an exception"
-                + " while shutting down.", t);
+    shutdown = false;
+    rescanImmediately = true;
+    scanTimeMs = 0;
+    LOG.info("Starting CacheReplicationMonitor with interval " +
+             intervalMs + " milliseconds");
+    try {
+      long curTimeMs = Time.monotonicNow();
+      while (true) {
+        synchronized(this) {
+          while (true) {
+            if (shutdown) {
+              LOG.info("Shutting down CacheReplicationMonitor");
+              return;
+            }
+            if (rescanImmediately) {
+              LOG.info("Rescanning on request");
+              rescanImmediately = false;
+              break;
+            }
+            long delta = (scanTimeMs + intervalMs) - curTimeMs;
+            if (delta <= 0) {
+              LOG.info("Rescanning after " + (curTimeMs - scanTimeMs) +
+                  " milliseconds");
+              break;
+            }
+            this.wait(delta);
+            curTimeMs = Time.monotonicNow();
           }
-          break;
         }
-        LOG.fatal("ReplicationMonitor thread received Runtime exception. ", t);
-        terminate(1, t);
+        scanTimeMs = curTimeMs;
+        mark = !mark;
+        rescan();
+        curTimeMs = Time.monotonicNow();
+        LOG.info("Scanned " + scannedDirectives + " directive(s) and " +
+            scannedBlocks + " block(s) in " + (curTimeMs - scanTimeMs) + " " +
+            "millisecond(s).");
       }
+    } catch (Throwable t) {
+      LOG.fatal("Thread exiting", t);
+      terminate(1, t);
     }
   }
 
   /**
-   * Assigns under-cached blocks to new datanodes.
+   * Kick the monitor thread.
+   * 
+   * If it is sleeping, it will wake up and start scanning.
+   * If it is currently scanning, it will finish the scan and immediately do 
+   * another one.
    */
-  private void computeCachingWork() {
-    List<Block> blocksToCache = null;
-    namesystem.writeLock();
+  public synchronized void kick() {
+    rescanImmediately = true;
+    this.notifyAll();
+  }
+
+  /**
+   * Shut down and join the monitor thread.
+   */
+  @Override
+  public void close() throws IOException {
+    synchronized(this) {
+      if (shutdown) return;
+      shutdown = true;
+      this.notifyAll();
+    }
     try {
-      synchronized (neededCacheBlocks) {
-        blocksToCache = neededCacheBlocks.pollAll();
+      if (this.isAlive()) {
+        this.join(60000);
       }
-    } finally {
-      namesystem.writeUnlock();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
     }
-    computeCachingWorkForBlocks(blocksToCache);
-    computeUncacheWork();
   }
 
-  private void computeCachingWorkForBlocks(List<Block> blocksToCache) {
-    int requiredRepl, effectiveRepl, additionalRepl;
-    List<DatanodeDescriptor> cachedNodes, storedNodes, targets;
-
-    final HashMap<Block, List<DatanodeDescriptor>> work =
-        new HashMap<Block, List<DatanodeDescriptor>>();
+  private void rescan() {
+    scannedDirectives = 0;
+    scannedBlocks = 0;
     namesystem.writeLock();
     try {
-      synchronized (neededCacheBlocks) {
-        for (Block block: blocksToCache) {
-          // Required number of cached replicas
-          requiredRepl = cacheReplManager.getCacheReplication(block);
-          // Replicas that are safely cached
-          cachedNodes = cacheReplManager.getSafeReplicas(
-              cacheReplManager.cachedBlocksMap, block);
-          // Replicas that are safely stored on disk
-          storedNodes = cacheReplManager.getSafeReplicas(
-              blockManager.blocksMap, block);
-          // "effective" replication factor which includes pending
-          // replication work
-          effectiveRepl = cachedNodes.size()
-              + pendingCacheBlocks.getNumReplicas(block);
-          if (effectiveRepl >= requiredRepl) {
-            neededCacheBlocks.remove(block);
-            blockLog.info("BLOCK* Removing " + block
-                + " from neededCacheBlocks as it has enough cached replicas");
-              continue;
-          }
-          // Choose some replicas to cache if needed
-          additionalRepl = requiredRepl - effectiveRepl;
-          targets = new ArrayList<DatanodeDescriptor>(storedNodes.size());
-          // Only target replicas that aren't already cached.
-          for (DatanodeDescriptor dn: storedNodes) {
-            if (!cachedNodes.contains(dn)) {
-              targets.add(dn);
-            }
-          }
-          if (targets.size() < additionalRepl) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Block " + block + " cannot be cached on additional"
-                  + " nodes because there are no more available datanodes"
-                  + " with the block on disk.");
-            }
-          }
-          targets = CacheReplicationPolicy.chooseTargetsToCache(block, targets,
-              additionalRepl);
-          if (targets.size() < additionalRepl) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Block " + block + " cannot be cached on additional"
-                  + " nodes because there is not sufficient cache space on"
-                  + " available target datanodes.");
-            }
-          }
-          // Continue if we couldn't get more cache targets
-          if (targets.size() == 0) {
-            continue;
-          }
-
-          // Update datanodes and blocks that were scheduled for caching
-          work.put(block, targets);
-          // Schedule caching on the targets
-          for (DatanodeDescriptor target: targets) {
-            target.addBlockToBeCached(block);
-          }
-          // Add block to the pending queue
-          pendingCacheBlocks.increment(block,
-              targets.toArray(new DatanodeDescriptor[] {}));
-          if (blockLog.isDebugEnabled()) {
-            blockLog.debug("BLOCK* block " + block
-                + " is moved from neededCacheBlocks to pendingCacheBlocks");
-          }
-          // Remove from needed queue if it will be fully replicated
-          if (effectiveRepl + targets.size() >= requiredRepl) {
-            neededCacheBlocks.remove(block);
-          }
-        }
-      }
+      rescanPathBasedCacheEntries();
     } finally {
       namesystem.writeUnlock();
     }
+    namesystem.writeLock();
+    try {
+      rescanCachedBlockMap();
+    } finally {
+      namesystem.writeUnlock();
+    }
+  }
 
-    if (blockLog.isInfoEnabled()) {
-      // log which blocks have been scheduled for replication
-      for (Entry<Block, List<DatanodeDescriptor>> item : work.entrySet()) {
-        Block block = item.getKey();
-        List<DatanodeDescriptor> nodes = item.getValue();
-        StringBuilder targetList = new StringBuilder("datanode(s)");
-        for (DatanodeDescriptor node: nodes) {
-          targetList.append(' ');
-          targetList.append(node);
+  /**
+   * Scan all PathBasedCacheEntries.  Use the information to figure out
+   * what cache replication factor each block should have.
+   *
+   * @param mark       Whether the current scan is setting or clearing the mark
+   */
+  private void rescanPathBasedCacheEntries() {
+    FSDirectory fsDir = namesystem.getFSDirectory();
+    for (PathBasedCacheEntry pce : cacheManager.getEntriesById().values()) {
+      scannedDirectives++;
+      String path = pce.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(pce, child.asFile());
+          }
+        }
+      } else if (node.isFile()) {
+        rescanFile(pce, node.asFile());
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Ignoring non-directory, non-file inode " + node +
+                    " found at " + path);
         }
-        blockLog.info("BLOCK* ask " + targetList + " to cache " + block);
       }
     }
-
-    if (blockLog.isDebugEnabled()) {
-        blockLog.debug(
-          "BLOCK* neededCacheBlocks = " + neededCacheBlocks.size()
-          + " pendingCacheBlocks = " + pendingCacheBlocks.size());
+  }
+  
+  /**
+   * Apply a PathBasedCacheEntry to a file.
+   *
+   * @param pce       The PathBasedCacheEntry to apply.
+   * @param file      The file.
+   */
+  private void rescanFile(PathBasedCacheEntry pce, INodeFile file) {
+    BlockInfo[] blockInfos = file.getBlocks();
+    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(),
+          pce.getReplication(), mark);
+      CachedBlock ocblock = cachedBlocks.get(ncblock);
+      if (ocblock == null) {
+        cachedBlocks.put(ncblock);
+      } else {
+        if (mark != ocblock.getMark()) {
+          // Mark hasn't been set in this scan, so update replication and mark.
+          ocblock.setReplicationAndMark(pce.getReplication(), mark);
+        } else {
+          // Mark already set in this scan.  Set replication to highest value in
+          // any PathBasedCacheEntry that covers this file.
+          ocblock.setReplicationAndMark((short)Math.max(
+              pce.getReplication(), ocblock.getReplication()), mark);
+        }
+      }
     }
   }
 
   /**
-   * Reassign pending caching work that has timed out
+   * 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 processPendingCachingWork() {
-    Block[] timedOutItems = pendingCacheBlocks.getTimedOutBlocks();
-    if (timedOutItems != null) {
-      namesystem.writeLock();
-      try {
-        for (int i = 0; i < timedOutItems.length; i++) {
-          Block block = timedOutItems[i];
-          final short numCached = cacheReplManager.getNumCached(block);
-          final short cacheReplication =
-              cacheReplManager.getCacheReplication(block);
-          // Needs to be cached if under-replicated
-          if (numCached < cacheReplication) {
-            synchronized (neededCacheBlocks) {
-              neededCacheBlocks.add(block);
-            }
-          }
+  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();
+        }
+      }
+      // If the block's mark doesn't match with the mark of this scan, that
+      // means that this block couldn't be reached during this scan.  That means
+      // it doesn't need to be cached any more.
+      int neededCached = (cblock.getMark() != mark) ?
+          0 : cblock.getReplication();
+      int numCached = cached.size();
+      if (numCached >= neededCached) {
+        // If we have enough replicas, drop all pending cached.
+        for (DatanodeDescriptor datanode : pendingCached) {
+          datanode.getPendingCached().remove(cblock);
+        }
+        pendingCached.clear();
+      }
+      if (numCached < neededCached) {
+        // If we don't have enough replicas, drop all pending uncached.
+        for (DatanodeDescriptor datanode : pendingUncached) {
+          datanode.getPendingUncached().remove(cblock);
+        }
+        pendingUncached.clear();
+      }
+      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);
         }
-      } finally {
-        namesystem.writeUnlock();
+      }
+      if ((neededCached == 0) &&
+          pendingUncached.isEmpty() &&
+          pendingCached.isEmpty()) {
+        // we have nothing more to do with this block.
+        cbIter.remove();
       }
     }
   }
 
   /**
-   * Schedule blocks for uncaching at datanodes
-   * @return total number of block for deletion
+   * 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.
    */
-  int computeUncacheWork() {
-    final List<String> nodes = blocksToUncache.getStorageIDs();
-    int blockCnt = 0;
-    for (String node: nodes) {
-      blockCnt += uncachingWorkForOneNode(node);
+  private void addNewPendingUncached(int neededUncached,
+      CachedBlock cachedBlock, List<DatanodeDescriptor> cached,
+      List<DatanodeDescriptor> pendingUncached) {
+    if (!cacheManager.isActive()) {
+      return;
+    }
+    // Figure out which replicas can be uncached.
+    LinkedList<DatanodeDescriptor> possibilities =
+        new LinkedList<DatanodeDescriptor>();
+    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--;
     }
-    return blockCnt;
   }
-
+  
   /**
-   * Gets the list of blocks scheduled for uncaching at a datanode and
-   * schedules them for uncaching.
-   * 
-   * @return number of blocks scheduled for removal
+   * 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 int uncachingWorkForOneNode(String nodeId) {
-    final List<Block> toInvalidate;
-    final DatanodeDescriptor dn;
-
-    namesystem.writeLock();
-    try {
-      // get blocks to invalidate for the nodeId
-      assert nodeId != null;
-      dn = datanodeManager.getDatanode(nodeId);
-      if (dn == null) {
-        blocksToUncache.remove(nodeId);
-        return 0;
+  private void addNewPendingCached(int neededCached,
+      CachedBlock cachedBlock, List<DatanodeDescriptor> cached,
+      List<DatanodeDescriptor> pendingCached) {
+    if (!cacheManager.isActive()) {
+      return;
+    }
+    // To figure out which replicas can be cached, we consult the
+    // blocksMap.  We don't want to try to cache a corrupt replica, though.
+    BlockInfo blockInfo = blockManager.
+          getStoredBlock(new Block(cachedBlock.getBlockId()));
+    if (blockInfo == null) {
+      LOG.debug("Not caching block " + cachedBlock + " because it " +
+          "was deleted from all DataNodes.");
+      return;
+    }
+    if (!blockInfo.isComplete()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not caching block " + cachedBlock + " because it " +
+            "is not yet complete.");
       }
-      toInvalidate = blocksToUncache.invalidateWork(nodeId, dn);
-      if (toInvalidate == null) {
-        return 0;
+      return;
+    }
+    List<DatanodeDescriptor> possibilities = new LinkedList<DatanodeDescriptor>();
+    int numReplicas = blockInfo.getCapacity();
+    Collection<DatanodeDescriptor> corrupt =
+        blockManager.getCorruptReplicas(blockInfo);
+    for (int i = 0; i < numReplicas; i++) {
+      DatanodeDescriptor datanode = blockInfo.getDatanode(i);
+      if ((datanode != null) && 
+          ((!pendingCached.contains(datanode)) &&
+          ((corrupt == null) || (!corrupt.contains(datanode))))) {
+        possibilities.add(datanode);
       }
-    } finally {
-      namesystem.writeUnlock();
     }
-    if (blockLog.isInfoEnabled()) {
-      blockLog.info("BLOCK* " + getClass().getSimpleName()
-          + ": ask " + dn + " to uncache " + toInvalidate);
+    while (neededCached > 0) {
+      if (possibilities.isEmpty()) {
+        LOG.warn("We need " + neededCached + " more replica(s) than " +
+            "actually exist to provide a cache replication of " +
+            cachedBlock.getReplication() + " for " + cachedBlock);
+        return;
+      }
+      DatanodeDescriptor datanode =
+          possibilities.remove(random.nextInt(possibilities.size()));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("AddNewPendingCached: datanode " + datanode + 
+            " will now cache block " + cachedBlock);
+      }
+      pendingCached.add(datanode);
+      boolean added = datanode.getPendingCached().add(cachedBlock);
+      assert added;
+      neededCached--;
     }
-    return toInvalidate.size();
   }
 }

+ 0 - 129
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationPolicy.java

@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-
-import org.apache.commons.math.random.RandomData;
-import org.apache.commons.math.random.RandomDataImpl;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.Block;
-
-/**
- * Helper class used by the CacheReplicationManager and CacheReplicationMonitor
- * to select datanodes where blocks should be cached or uncached.
- */
-@InterfaceAudience.LimitedPrivate({"HDFS"})
-public class CacheReplicationPolicy {
-
-  // Not thread-safe, but only accessed by the CacheReplicationMonitor
-  private static RandomData random = new RandomDataImpl();
-
-  /**
-   * @return List of datanodes with sufficient capacity to cache the block
-   */
-  private static List<DatanodeDescriptor> selectSufficientCapacity(Block block,
-      List<DatanodeDescriptor> targets) {
-    List<DatanodeDescriptor> sufficient =
-        new ArrayList<DatanodeDescriptor>(targets.size());
-    for (DatanodeDescriptor dn: targets) {
-      long remaining = dn.getCacheRemaining();
-      if (remaining >= block.getNumBytes()) {
-        sufficient.add(dn);
-      }
-    }
-    return sufficient;
-  }
-
-  /**
-   * Returns a random datanode from targets, weighted by the amount of free
-   * cache capacity on the datanode.
-   * 
-   * @param block Block to be cached
-   * @param targets List of potential cache targets
-   * @return a random DN, or null if no datanodes are available or have enough
-   *         cache capacity.
-   */
-  private static DatanodeDescriptor randomDatanodeByRemainingCache(Block block,
-      List<DatanodeDescriptor> targets) {
-    // Hold a lottery biased by the amount of free space to decide
-    // who gets the block
-    Collections.shuffle(targets);
-    TreeMap<Long, DatanodeDescriptor> lottery =
-        new TreeMap<Long, DatanodeDescriptor>();
-    long totalCacheAvailable = 0;
-    for (DatanodeDescriptor dn: targets) {
-      long remaining = dn.getCacheRemaining();
-      totalCacheAvailable += remaining;
-      lottery.put(totalCacheAvailable, dn);
-    }
-    // Pick our lottery winner
-    long winningTicket = random.nextLong(0, totalCacheAvailable - 1);
-    Entry<Long, DatanodeDescriptor> winner = lottery.higherEntry(winningTicket);
-    return winner.getValue();
-  }
-
-  /**
-   * Chooses numTargets new cache replicas for a block from a list of targets.
-   * Will return fewer targets than requested if not enough nodes are available.
-   * 
-   * @return List of target datanodes
-   */
-  static List<DatanodeDescriptor> chooseTargetsToCache(Block block,
-      List<DatanodeDescriptor> targets, int numTargets) {
-    List<DatanodeDescriptor> sufficient =
-        selectSufficientCapacity(block, targets);
-    List<DatanodeDescriptor> chosen =
-        new ArrayList<DatanodeDescriptor>(numTargets);
-    for (int i = 0; i < numTargets && !sufficient.isEmpty(); i++) {
-      DatanodeDescriptor choice =
-          randomDatanodeByRemainingCache(block, sufficient);
-      chosen.add(choice);
-      sufficient.remove(choice);
-    }
-    return chosen;
-  }
-
-  /**
-   * Given a list cache replicas where a block is cached, choose replicas to
-   * uncache to drop the cache replication factor down to replication.
-   * 
-   * @param nodes list of datanodes where the block is currently cached
-   * @param replication desired replication factor
-   * @return List of datanodes to uncache
-   */
-  public static List<DatanodeDescriptor> chooseTargetsToUncache(
-      List<DatanodeDescriptor> nodes, short replication) {
-    final int effectiveReplication = nodes.size();
-    List<DatanodeDescriptor> targets =
-        new ArrayList<DatanodeDescriptor>(effectiveReplication);
-    Collections.shuffle(nodes);
-    final int additionalTargetsNeeded = effectiveReplication - replication;
-    int chosen = 0;
-    while (chosen < additionalTargetsNeeded && !nodes.isEmpty()) {
-      targets.add(nodes.get(chosen));
-      chosen++;
-    }
-    return targets;
-  }
-
-}

+ 70 - 121
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -28,7 +28,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
+import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -98,22 +100,72 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
-   * Head of the list of blocks on the datanode
+   * A list of CachedBlock objects on this datanode.
    */
-  private volatile BlockInfo blockList = null;
+  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;
+    }
+  }
+
   /**
-   * Number of blocks on the datanode
+   * The blocks which we want to cache on this DataNode.
    */
-  private int numBlocks = 0;
+  private final CachedBlocksList pendingCached = 
+      new CachedBlocksList(this, CachedBlocksList.Type.PENDING_CACHED);
 
   /**
-   * Head of the list of cached blocks on the datanode
+   * The blocks which we know are cached on this datanode.
+   * This list is updated by periodic cache reports.
    */
-  private volatile BlockInfo cachedBlockList = null;
+  private final CachedBlocksList cached = 
+      new CachedBlocksList(this, CachedBlocksList.Type.CACHED);
+
   /**
-   * Number of cached blocks on the datanode
+   * The blocks which we want to uncache on this DataNode.
    */
-  private int numCachedBlocks = 0;
+  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;
+  }
+
+  /**
+   * Head of the list of blocks on the datanode
+   */
+  private volatile BlockInfo blockList = null;
+  /**
+   * Number of blocks on the datanode
+   */
+  private int numBlocks = 0;
 
   // isAlive == heartbeats.contains(this)
   // This is an optimization, because contains takes O(n) time on Arraylist
@@ -154,12 +206,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** A set of blocks to be invalidated by this datanode */
   private LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
 
-  /** A queue of blocks to be cached by this datanode */
-  private BlockQueue<Block> cacheBlocks = new BlockQueue<Block>();
-  /** A set of blocks to be uncached by this datanode */
-  private LightWeightHashSet<Block> blocksToUncache =
-      new LightWeightHashSet<Block>();
-
   /* Variables for maintaining number of blocks scheduled to be written to
    * this datanode. This count is approximate and might be slightly bigger
    * in case of errors (e.g. datanode does not report if an error occurs
@@ -286,43 +332,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return curIndex;
   }
 
-  /**
-   * Add block to the list of cached blocks on the data-node.
-   * @return true if block was successfully added, false if already present
-   */
-  public boolean addCachedBlock(BlockInfo b) {
-    if (!b.addNode(this))
-      return false;
-    // add to the head of the data-node list
-    cachedBlockList = b.listInsert(cachedBlockList, this);
-    numCachedBlocks++;
-    return true;
-  }
-
-  /**
-   * Remove block from the list of cached blocks on the data-node.
-   * @return true if block was successfully removed, false if not present
-   */
-  public boolean removeCachedBlock(BlockInfo b) {
-    cachedBlockList = b.listRemove(cachedBlockList, this);
-    if (b.removeNode(this)) {
-      numCachedBlocks--;
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   * Move block to the head of the list of cached blocks on the data-node.
-   * @return the index of the head of the blockList
-   */
-  int moveCachedBlockToHead(BlockInfo b, int curIndex, int headIndex) {
-    cachedBlockList = b.moveBlockToHead(cachedBlockList, this, curIndex,
-        headIndex);
-    return curIndex;
-  }
-
   /**
    * Used for testing only
    * @return the head of the blockList
@@ -332,11 +341,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return blockList;
   }
 
-  @VisibleForTesting
-  protected BlockInfo getCachedHead() {
-    return cachedBlockList;
-  }
-
   /**
    * Replace specified old block with a new one in the DataNodeDescriptor.
    *
@@ -359,10 +363,13 @@ public class DatanodeDescriptor extends DatanodeInfo {
     setDfsUsed(0);
     setXceiverCount(0);
     this.blockList = null;
-    this.cachedBlockList = null;
     this.invalidateBlocks.clear();
-    this.blocksToUncache.clear();
     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() {
@@ -371,20 +378,17 @@ public class DatanodeDescriptor extends DatanodeInfo {
       this.recoverBlocks.clear();
       this.replicateBlocks.clear();
     }
-    synchronized(blocksToUncache) {
-      this.blocksToUncache.clear();
-      this.cacheBlocks.clear();
-    }
+    // pendingCached, cached, and pendingUncached are protected by the
+    // FSN lock.
+    this.pendingCached.clear();
+    this.cached.clear();
+    this.pendingUncached.clear();
   }
 
   public int numBlocks() {
     return numBlocks;
   }
 
-  public int numCachedBlocks() {
-    return numCachedBlocks;
-  }
-
   /**
    * Updates stats from datanode heartbeat.
    */
@@ -438,10 +442,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return new BlockIterator(this.blockList, this);
   }
 
-  public Iterator<BlockInfo> getCachedBlockIterator() {
-    return new BlockIterator(this.cachedBlockList, this);
-  }
-
   /**
    * Store block replication work.
    */
@@ -450,14 +450,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     replicateBlocks.offer(new BlockTargetPair(block, targets));
   }
 
-  /**
-   * Store block caching work.
-   */
-  void addBlockToBeCached(Block block) {
-    assert(block != null);
-    cacheBlocks.offer(block);
-  }
-
   /**
    * Store block recovery work.
    */
@@ -482,18 +474,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
   
-  /**
-   * Store block uncaching work.
-   */
-  void addBlocksToBeUncached(List<Block> blocklist) {
-    assert(blocklist != null && blocklist.size() > 0);
-    synchronized (blocksToUncache) {
-      for (Block blk : blocklist) {
-        blocksToUncache.add(blk);
-      }
-    }
-  }
-
   /**
    * The number of work items that are pending to be replicated
    */
@@ -501,13 +481,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return replicateBlocks.size();
   }
 
-  /**
-   * The number of pending cache work items
-   */
-  int getNumberOfBlocksToBeCached() {
-    return cacheBlocks.size();
-  }
-
   /**
    * The number of block invalidation items that are pending to 
    * be sent to the datanode
@@ -518,23 +491,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  /**
-   * The number of pending uncache work items
-   */
-  int getNumberOfBlocksToBeUncached() {
-    synchronized (blocksToUncache) {
-      return blocksToUncache.size();
-    }
-  }
-
   public List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
     return replicateBlocks.poll(maxTransfers);
   }
 
-  public List<Block> getCacheBlocks() {
-    return cacheBlocks.poll(cacheBlocks.size());
-  }
-
   public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
     List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
@@ -553,17 +513,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  /**
-   * Remove up to the maximum number of blocks to be uncached
-   */
-  public Block[] getInvalidateCacheBlocks() {
-    synchronized (blocksToUncache) {
-      Block[] deleteList = blocksToUncache.pollToArray(
-          new Block[blocksToUncache.size()]);
-      return deleteList.length == 0 ? null : deleteList;
-    }
-  }
-
   /**
    * @return Approximate number of blocks currently scheduled to be written 
    * to this datanode.

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

@@ -49,6 +49,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 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.Entry;
 import org.apache.hadoop.hdfs.server.namenode.HostFileManager.EntrySet;
@@ -76,6 +78,7 @@ import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.ScriptBasedMapping;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Time;
 
@@ -167,6 +170,12 @@ public class DatanodeManager {
    */
   private boolean hasClusterEverBeenMultiRack = false;
 
+  /**
+   * Whether we should tell datanodes what to cache in replies to
+   * heartbeat messages.
+   */
+  private boolean sendCachingCommands = false;
+
   /**
    * The number of datanodes for each software version. This list should change
    * during rolling upgrades.
@@ -1305,26 +1314,17 @@ public class DatanodeManager {
           cmds.add(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE,
               blockPoolId, blks));
         }
-        
-        // Check pending caching
-        List<Block> pendingCacheList = nodeinfo.getCacheBlocks();
-        if (pendingCacheList != null) {
-          long blockIds[] = new long[pendingCacheList.size()];
-          for (int i = 0; i < pendingCacheList.size(); i++) {
-            blockIds[i] = pendingCacheList.get(i).getBlockId();
-          }
-          cmds.add(new BlockIdCommand(DatanodeProtocol.DNA_CACHE, blockPoolId,
-              blockIds));
+        DatanodeCommand pendingCacheCommand =
+            getCacheCommand(nodeinfo.getPendingCached(), nodeinfo,
+              DatanodeProtocol.DNA_CACHE, blockPoolId);
+        if (pendingCacheCommand != null) {
+          cmds.add(pendingCacheCommand);
         }
-        // Check cached block invalidation
-        blks = nodeinfo.getInvalidateCacheBlocks();
-        if (blks != null) {
-          long blockIds[] = new long[blks.length];
-          for (int i = 0; i < blks.length; i++) {
-            blockIds[i] = blks[i].getBlockId();
-          }
-          cmds.add(new BlockIdCommand(DatanodeProtocol.DNA_UNCACHE,
-              blockPoolId, blockIds));
+        DatanodeCommand pendingUncacheCommand =
+            getCacheCommand(nodeinfo.getPendingUncached(), nodeinfo,
+              DatanodeProtocol.DNA_UNCACHE, blockPoolId);
+        if (pendingUncacheCommand != null) {
+          cmds.add(pendingUncacheCommand);
         }
 
         blockManager.addKeyUpdateCommand(cmds, nodeinfo);
@@ -1345,6 +1345,40 @@ public class DatanodeManager {
     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 and clear 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();
+      iter.remove();
+    }
+    if (!sendCachingCommands) {
+      // Do not send caching commands unless the FSNamesystem told us we
+      // should.
+      return null;
+    }
+    return new BlockIdCommand(action, poolId, blockIds);
+  }
+
   /**
    * Tell all datanodes to use a new, non-persistent bandwidth value for
    * dfs.balance.bandwidthPerSec.
@@ -1393,4 +1427,8 @@ public class DatanodeManager {
   public String toString() {
     return getClass().getSimpleName() + ": " + host2DatanodeMap;
   }
+
+  public void setSendCachingCommands(boolean sendCachingCommands) {
+    this.sendCachingCommands = sendCachingCommands;
+  }
 }

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

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -34,22 +35,24 @@ import org.apache.hadoop.hdfs.util.LightWeightHashSet;
  * on the machine in question.
  */
 @InterfaceAudience.Private
-abstract class InvalidateBlocks {
+class InvalidateBlocks {
   /** Mapping: StorageID -> Collection of Blocks */
   private final Map<String, LightWeightHashSet<Block>> node2blocks =
       new TreeMap<String, LightWeightHashSet<Block>>();
   /** The total number of blocks in the map. */
   private long numBlocks = 0L;
 
+  private final DatanodeManager datanodeManager;
+
+  InvalidateBlocks(final DatanodeManager datanodeManager) {
+    this.datanodeManager = datanodeManager;
+  }
+
   /** @return the number of blocks to be invalidated . */
   synchronized long numBlocks() {
     return numBlocks;
   }
 
-  synchronized int numStorages() {
-    return node2blocks.size();
-  }
-
   /**
    * @return true if the given storage has the given block listed for
    * invalidation. Blocks are compared including their generation stamps:
@@ -108,22 +111,22 @@ abstract class InvalidateBlocks {
     }
   }
 
-  /**
-   * Polls up to <i>limit</i> blocks from the list of to-be-invalidated Blocks
-   * for a storage.
-   */
-  synchronized List<Block> pollNumBlocks(final String storageId, final int limit) {
-    final LightWeightHashSet<Block> set = node2blocks.get(storageId);
-    if (set == null) {
-      return null;
+  /** Print the contents to out. */
+  synchronized void dump(final PrintWriter out) {
+    final int size = node2blocks.values().size();
+    out.println("Metasave: Blocks " + numBlocks 
+        + " waiting deletion from " + size + " datanodes.");
+    if (size == 0) {
+      return;
     }
-    List<Block> polledBlocks = set.pollN(limit);
-    // Remove the storage if the set is now empty
-    if (set.isEmpty()) {
-      remove(storageId);
+
+    for(Map.Entry<String,LightWeightHashSet<Block>> entry : node2blocks.entrySet()) {
+      final LightWeightHashSet<Block> blocks = entry.getValue();
+      if (blocks.size() > 0) {
+        out.println(datanodeManager.getDatanode(entry.getKey()));
+        out.println(blocks);
+      }
     }
-    numBlocks -= polledBlocks.size();
-    return polledBlocks;
   }
 
   /** @return a list of the storage IDs. */
@@ -131,22 +134,26 @@ abstract class InvalidateBlocks {
     return new ArrayList<String>(node2blocks.keySet());
   }
 
-  /**
-   * Return the set of to-be-invalidated blocks for a storage.
-   */
-  synchronized LightWeightHashSet<Block> getBlocks(String storageId) {
-    return node2blocks.get(storageId);
-  }
+  synchronized List<Block> invalidateWork(
+      final String storageId, final DatanodeDescriptor dn) {
+    final LightWeightHashSet<Block> set = node2blocks.get(storageId);
+    if (set == null) {
+      return null;
+    }
 
-  /**
-   * Schedules invalidation work associated with a storage at the corresponding
-   * datanode.
-   * @param storageId Storage of blocks to be invalidated
-   * @param dn Datanode where invalidation work will be scheduled
-   * @return List of blocks scheduled for invalidation at the datanode
-   */
-  abstract List<Block> invalidateWork(final String storageId,
-      final DatanodeDescriptor dn);
+    // # blocks that can be sent in one message is limited
+    final int limit = datanodeManager.blockInvalidateLimit;
+    final List<Block> toInvalidate = set.pollN(limit);
+
+    // If we send everything in this message, remove this node entry
+    if (set.isEmpty()) {
+      remove(storageId);
+    }
+
+    dn.addBlocksToBeInvalidated(toInvalidate);
+    numBlocks -= toInvalidate.size();
+    return toInvalidate;
+  }
   
   synchronized void clear() {
     node2blocks.clear();

+ 0 - 67
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateStoredBlocks.java

@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.PrintWriter;
-import java.util.List;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.util.LightWeightHashSet;
-
-/**
- * Subclass of InvalidateBlocks used by the BlockManager to
- * track blocks on each storage that are scheduled to be invalidated.
- */
-public class InvalidateStoredBlocks extends InvalidateBlocks {
-
-  private final DatanodeManager datanodeManager;
-
-  InvalidateStoredBlocks(DatanodeManager datanodeManager) {
-    this.datanodeManager = datanodeManager;
-  }
-
-  /** Print the contents to out. */
-  synchronized void dump(final PrintWriter out) {
-    final int size = numStorages();
-    out.println("Metasave: Blocks " + numBlocks() 
-        + " waiting deletion from " + size + " datanodes.");
-    if (size == 0) {
-      return;
-    }
-
-    List<String> storageIds = getStorageIDs();
-    for (String storageId: storageIds) {
-      LightWeightHashSet<Block> blocks = getBlocks(storageId);
-      if (blocks != null && !blocks.isEmpty()) {
-        out.println(datanodeManager.getDatanode(storageId));
-        out.println(blocks);
-      }
-    }
-  }
-
-  @Override
-  synchronized List<Block> invalidateWork(
-      final String storageId, final DatanodeDescriptor dn) {
-    final List<Block> toInvalidate = pollNumBlocks(storageId,
-        datanodeManager.blockInvalidateLimit);
-    if (toInvalidate != null) {
-      dn.addBlocksToBeInvalidated(toInvalidate);
-    }
-    return toInvalidate;
-  }
-}

+ 11 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java

@@ -29,27 +29,20 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.util.Daemon;
 
-/**
- * PendingReplicationBlocks is used in the BlockManager to track blocks that are
- * currently being replicated on disk and in the CacheReplicationManager to
- * track blocks that are currently being cached.
- * 
- * <p>
- * PendingReplicationBlocks performs the following tasks:
- * </p>
- * 
- * <ol>
- * <li>tracks in-flight replication or caching requests for a block at target
- * datanodes.</li>
- * <li>identifies requests that have timed out and need to be rescheduled at a
- * different datanode.</li>
- * </ol>
- */
-@InterfaceAudience.LimitedPrivate({"HDFS"})
+/***************************************************
+ * PendingReplicationBlocks does the bookkeeping of all
+ * blocks that are getting replicated.
+ *
+ * It does the following:
+ * 1)  record blocks that are getting replicated at this instant.
+ * 2)  a coarse grain timer to track age of replication request
+ * 3)  a thread that periodically identifies replication-requests
+ *     that never made it.
+ *
+ ***************************************************/
 class PendingReplicationBlocks {
   private static final Log LOG = BlockManager.LOG;
 

+ 0 - 271
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReportProcessor.java

@@ -1,271 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedList;
-
-import org.apache.commons.logging.Log;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Handles common operations of processing a block report from a datanode,
- * generating a diff of updates to the BlocksMap, and then feeding the diff
- * to the subclass-implemented hooks.
- */
-@InterfaceAudience.LimitedPrivate({"HDFS"})
-public abstract class ReportProcessor {
-
-  static final Log blockLog = NameNode.blockStateChangeLog;
-  private final String className = getClass().getSimpleName();
-  // Max number of blocks to log info about during a block report.
-  final long maxNumBlocksToLog;
-
-  void blockLogDebug(String message) {
-    if (blockLog.isDebugEnabled()) {
-      blockLog.info("BLOCK* " + className + message);
-    }
-  }
-
-  void blockLogInfo(String message) {
-    if (blockLog.isInfoEnabled()) {
-      blockLog.info("BLOCK* " + className + message);
-    }
-  }
-
-  void blockLogWarn(String message) {
-    blockLog.warn("BLOCK* " + className + message);
-  }
-
-  void logAddStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) {
-    if (!blockLog.isInfoEnabled()) {
-      return;
-    }
-    StringBuilder sb = new StringBuilder(500);
-    sb.append("BLOCK* " + className + "#addStoredBlock: blockMap updated: ")
-      .append(node)
-      .append(" is added to ");
-    storedBlock.appendStringTo(sb);
-    sb.append(" size " )
-      .append(storedBlock.getNumBytes());
-    blockLog.info(sb);
-  }
-
-  public ReportProcessor(Configuration conf) {
-    this.maxNumBlocksToLog = conf.getLong(
-        DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
-        DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT);
-  }
-
-  /**
-   * Processes a block report from a datanode, updating the block to
-   * datanode mapping, adding new blocks and removing invalid ones.
-   * Also computes and queues new replication and invalidation work.
-   * @param node Datanode sending the block report
-   * @param report as list of longs
-   * @throws IOException
-   */
-  final void processReport(final DatanodeDescriptor node,
-      final BlockListAsLongs report) throws IOException {
-    // Normal case:
-    // Modify the (block-->datanode) map, according to the difference
-    // between the old and new block report.
-    //
-    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
-    Collection<Block> toRemove = new LinkedList<Block>();
-    Collection<Block> toInvalidate = new LinkedList<Block>();
-    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
-    Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
-    reportDiff(node, report, toAdd, toRemove, toInvalidate, toCorrupt, toUC);
-
-    // Process the blocks on each queue
-    for (StatefulBlockInfo b : toUC) {
-      addStoredBlockUnderConstruction(b.storedBlock, node, b.reportedState);
-    }
-    for (Block b : toRemove) {
-      removeStoredBlock(b, node);
-    }
-    int numBlocksLogged = 0;
-    for (BlockInfo b : toAdd) {
-      addStoredBlock(b, node, null, numBlocksLogged < maxNumBlocksToLog);
-      numBlocksLogged++;
-    }
-
-    if (numBlocksLogged > maxNumBlocksToLog) {
-      blockLogInfo("#processReport: logged"
-          + " info for " + maxNumBlocksToLog
-          + " of " + numBlocksLogged + " reported.");
-    }
-    for (Block b : toInvalidate) {
-      blockLogInfo("#processReport: "
-          + b + " on " + node + " size " + b.getNumBytes()
-          + " does not belong to any file");
-      addToInvalidates(b, node);
-    }
-    for (BlockToMarkCorrupt b : toCorrupt) {
-      markBlockAsCorrupt(b, node);
-    }
-  }
-
-  /**
-   * Compute the difference between the current state of the datanode in the
-   * BlocksMap and the new reported state, categorizing changes into
-   * different groups (e.g. new blocks to be added, blocks that were removed,
-   * blocks that should be invalidated, etc.).
-   */
-  private void reportDiff(DatanodeDescriptor dn,
-      BlockListAsLongs newReport,
-      Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
-      Collection<Block> toRemove,           // remove from DatanodeDescriptor
-      Collection<Block> toInvalidate,       // should be removed from DN
-      Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
-      Collection<StatefulBlockInfo> toUC) { // add to under-construction list
-    // place a delimiter in the list which separates blocks
-    // that have been reported from those that have not
-    BlockInfo delimiter = new BlockInfo(new Block(), 1);
-    boolean added = addBlock(dn, delimiter);
-    assert added : "Delimiting block cannot be present in the node";
-    int headIndex = 0; //currently the delimiter is in the head of the list
-    int curIndex;
-
-    if (newReport == null) {
-      newReport = new BlockListAsLongs();
-    }
-    // scan the report and process newly reported blocks
-    BlockReportIterator itBR = newReport.getBlockReportIterator();
-    while (itBR.hasNext()) {
-      Block iblk = itBR.next();
-      ReplicaState iState = itBR.getCurrentReplicaState();
-      BlockInfo storedBlock = processReportedBlock(dn, iblk, iState,
-                                  toAdd, toInvalidate, toCorrupt, toUC);
-      // move block to the head of the list
-      if (storedBlock != null && (curIndex = storedBlock.findDatanode(dn)) >= 0) {
-        headIndex = moveBlockToHead(dn, storedBlock, curIndex, headIndex);
-      }
-    }
-    // collect blocks that have not been reported
-    // all of them are next to the delimiter
-    Iterator<? extends Block> it = new DatanodeDescriptor.BlockIterator(
-        delimiter.getNext(0), dn);
-    while (it.hasNext()) {
-      toRemove.add(it.next());
-    }
-    removeBlock(dn, delimiter);
-  }
-
-  // Operations on the blocks on a datanode
-
-  abstract int moveBlockToHead(DatanodeDescriptor dn, BlockInfo storedBlock,
-      int curIndex, int headIndex);
-
-  abstract boolean addBlock(DatanodeDescriptor dn, BlockInfo block);
-
-  abstract boolean removeBlock(DatanodeDescriptor dn, BlockInfo block);
-
-  // Cache report processing
-
-  abstract BlockInfo processReportedBlock(DatanodeDescriptor dn, Block iblk,
-      ReplicaState iState, Collection<BlockInfo> toAdd,
-      Collection<Block> toInvalidate, Collection<BlockToMarkCorrupt> toCorrupt,
-      Collection<StatefulBlockInfo> toUC);
-
-  // Hooks for processing the cache report diff
-
-  abstract Block addStoredBlock(final BlockInfo block,
-      DatanodeDescriptor node, DatanodeDescriptor delNodeHint,
-      boolean logEveryBlock) throws IOException;
-
-  abstract void removeStoredBlock(Block block, DatanodeDescriptor node);
-
-  abstract void markBlockAsCorrupt(BlockToMarkCorrupt b, DatanodeInfo dn)
-      throws IOException;
-
-  abstract void addToInvalidates(final Block b, final DatanodeInfo node);
-
-  abstract void addStoredBlockUnderConstruction(
-      BlockInfoUnderConstruction storedBlock, DatanodeDescriptor node,
-      ReplicaState reportedState) throws IOException;
-
-  /**
-   * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
-   * list of blocks that should be considered corrupt due to a block report.
-   */
-  static class BlockToMarkCorrupt {
-    /** The corrupted block in a datanode. */
-    final BlockInfo corrupted;
-    /** The corresponding block stored in the BlockManager. */
-    final BlockInfo stored;
-    /** The reason to mark corrupt. */
-    final String reason;
-
-    BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason) {
-      Preconditions.checkNotNull(corrupted, "corrupted is null");
-      Preconditions.checkNotNull(stored, "stored is null");
-
-      this.corrupted = corrupted;
-      this.stored = stored;
-      this.reason = reason;
-    }
-
-    BlockToMarkCorrupt(BlockInfo stored, String reason) {
-      this(stored, stored, reason);
-    }
-
-    BlockToMarkCorrupt(BlockInfo stored, long gs, String reason) {
-      this(new BlockInfo(stored), stored, reason);
-      //the corrupted block in datanode has a different generation stamp
-      corrupted.setGenerationStamp(gs);
-    }
-
-    @Override
-    public String toString() {
-      return corrupted + "("
-          + (corrupted == stored? "same as stored": "stored=" + stored) + ")";
-    }
-  }
-
-  /**
-   * StatefulBlockInfo is used to build the "toUC" list, which is a list of
-   * updates to the information about under-construction blocks.
-   * Besides the block in question, it provides the ReplicaState
-   * reported by the datanode in the block report.
-   */
-  static class StatefulBlockInfo {
-    final BlockInfoUnderConstruction storedBlock;
-    final ReplicaState reportedState;
-
-    StatefulBlockInfo(BlockInfoUnderConstruction storedBlock,
-        ReplicaState reportedState) {
-      this.storedBlock = storedBlock;
-      this.reportedState = reportedState;
-    }
-  }
-
-}

+ 0 - 44
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UncacheBlocks.java

@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.Block;
-
-/**
- * Subclass of InvalidateBlocks used by the CacheReplicationManager to
- * track blocks on each storage that are scheduled to be uncached.
- */
-@InterfaceAudience.Private
-public class UncacheBlocks extends InvalidateBlocks {
-
-  UncacheBlocks() {
-  }
-
-  @Override
-  synchronized List<Block> invalidateWork(
-      final String storageId, final DatanodeDescriptor dn) {
-    final List<Block> toInvalidate = pollNumBlocks(storageId, Integer.MAX_VALUE);
-    if (toInvalidate != null) {
-      dn.addBlocksToBeUncached(toInvalidate);
-    }
-    return toInvalidate;
-  }
-}

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

@@ -628,6 +628,8 @@ class BPOfferService {
     case DatanodeProtocol.DNA_FINALIZE:
     case DatanodeProtocol.DNA_RECOVERBLOCK:
     case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
+    case DatanodeProtocol.DNA_CACHE:
+    case DatanodeProtocol.DNA_UNCACHE:
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
       break;
     default:

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

@@ -21,24 +21,10 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CompletionService;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-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;
@@ -50,10 +36,6 @@ import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * Manages caching for an FsDatasetImpl by using the mmap(2) and mlock(2)

+ 331 - 178
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -17,53 +17,97 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_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_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_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.Closeable;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 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 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.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPoolNameError;
-import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.PoolWritePermissionDeniedError;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
+import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
+import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPoolNameError;
+import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.UnexpectedAddPathBasedCacheDirectiveException;
+import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.PoolWritePermissionDeniedError;
 import org.apache.hadoop.hdfs.protocol.PathBasedCacheEntry;
 import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.InvalidIdException;
 import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.NoSuchIdException;
-import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.RemovePermissionDeniedException;
 import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.UnexpectedRemovePathBasedCacheDescriptorException;
+import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.RemovePermissionDeniedException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+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.common.HdfsServerConstants.ReplicaState;
+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.io.Text;
+import org.apache.hadoop.util.GSet;
+import org.apache.hadoop.util.LightWeightGSet;
+import org.apache.hadoop.util.Time;
 
-import com.google.common.base.Preconditions;
+import com.google.common.annotations.VisibleForTesting;
 
 /**
  * The Cache Manager handles caching on DataNodes.
+ *
+ * This class is instantiated by the FSNamesystem when caching is enabled.
+ * 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);
 
+  // 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 entries, sorted by ID.
    *
@@ -73,6 +117,12 @@ public final class CacheManager {
   private final TreeMap<Long, PathBasedCacheEntry> entriesById =
       new TreeMap<Long, PathBasedCacheEntry>();
 
+  /**
+   * The entry ID to use for a new entry.  Entry IDs always increase, and are
+   * never reused.
+   */
+  private long nextEntryId;
+
   /**
    * Cache entries, sorted by path
    */
@@ -85,11 +135,6 @@ public final class CacheManager {
   private final TreeMap<String, CachePool> cachePools =
       new TreeMap<String, CachePool>();
 
-  /**
-   * The entry ID to use for a new entry.
-   */
-  private long nextEntryId;
-
   /**
    * Maximum number of cache pools to list in one operation.
    */
@@ -100,44 +145,129 @@ public final class CacheManager {
    */
   private final int maxListCacheDescriptorsResponses;
 
-  final private FSNamesystem namesystem;
-  final private FSDirectory dir;
+  /**
+   * Interval between scans in milliseconds.
+   */
+  private final long scanIntervalMs;
+
+  /**
+   * Whether caching is enabled.
+   *
+   * If caching is disabled, we will not process cache reports or store
+   * information about what is cached where.  We also do not start the
+   * CacheReplicationMonitor thread.  This will save resources, but provide
+   * less functionality.
+   *     
+   * Even when caching is disabled, we still store path-based cache
+   * information.  This information is stored in the edit log and fsimage.  We
+   * don't want to lose it just because a configuration setting was turned off.
+   * However, we will not act on this information if caching is disabled.
+   */
+  private final boolean enabled;
+
+  /**
+   * Whether the CacheManager is active.
+   * 
+   * When the CacheManager is active, it tells the DataNodes what to cache
+   * and uncache.  The CacheManager cannot become active if enabled = false.
+   */
+  private boolean active = false;
 
-  CacheManager(FSNamesystem namesystem, FSDirectory dir, Configuration conf) {
-    clear();
+  /**
+   * All cached blocks.
+   */
+  private final GSet<CachedBlock, CachedBlock> cachedBlocks;
+
+  /**
+   * The CacheReplicationMonitor.
+   */
+  private CacheReplicationMonitor monitor;
+
+  CacheManager(FSNamesystem namesystem, Configuration conf,
+      BlockManager blockManager) {
     this.namesystem = namesystem;
-    this.dir = dir;
-    maxListCachePoolsResponses = conf.getInt(
+    this.blockManager = blockManager;
+    this.nextEntryId = 1;
+    this.maxListCachePoolsResponses = conf.getInt(
         DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES,
         DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT);
-    maxListCacheDescriptorsResponses = conf.getInt(
+    this.maxListCacheDescriptorsResponses = conf.getInt(
         DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES,
         DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES_DEFAULT);
+    scanIntervalMs = conf.getLong(
+        DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS,
+        DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT);
+    this.enabled = conf.getBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY,
+        DFS_NAMENODE_CACHING_ENABLED_DEFAULT);
+    this.cachedBlocks = !enabled ? null :
+        new LightWeightGSet<CachedBlock, CachedBlock>(
+            LightWeightGSet.computeCapacity(0.25, "cachedBlocks"));
   }
 
-  synchronized void clear() {
-    entriesById.clear();
-    entriesByPath.clear();
-    cachePools.clear();
-    nextEntryId = 1;
+  /**
+   * Activate the cache manager.
+   * 
+   * When the cache manager is active, tell the datanodes where to cache files.
+   */
+  public void activate() {
+    assert namesystem.hasWriteLock();
+    if (enabled && (!active)) {
+      LOG.info("Activating CacheManager.  " +
+          "Starting replication monitor thread...");
+      active = true;
+      monitor = new CacheReplicationMonitor(namesystem, this,
+         scanIntervalMs);
+      monitor.start();
+    }
   }
 
   /**
-   * Returns the next entry ID to be used for a PathBasedCacheEntry
+   * Deactivate the cache manager.
+   * 
+   * When the cache manager is inactive, it does not tell the datanodes where to
+   * cache files.
    */
-  synchronized long getNextEntryId() {
-    Preconditions.checkArgument(nextEntryId != Long.MAX_VALUE);
-    return nextEntryId++;
+  public void deactivate() {
+    assert namesystem.hasWriteLock();
+    if (active) {
+      LOG.info("Deactivating CacheManager.  " +
+          "stopping CacheReplicationMonitor thread...");
+      active = false;
+      IOUtils.closeQuietly(monitor);
+      monitor = null;
+      LOG.info("CacheReplicationMonitor thread stopped and deactivated.");
+    }
   }
 
   /**
-   * Returns the PathBasedCacheEntry corresponding to a PathBasedCacheEntry.
-   * 
-   * @param directive Lookup directive
-   * @return Corresponding PathBasedCacheEntry, or null if not present.
+   * Return true only if the cache manager is active.
+   * Must be called under the FSN read or write lock.
    */
-  private synchronized PathBasedCacheEntry
-      findEntry(PathBasedCacheDirective directive) {
+  public boolean isActive() {
+    return active;
+  }
+
+  public TreeMap<Long, PathBasedCacheEntry> getEntriesById() {
+    assert namesystem.hasReadOrWriteLock();
+    return entriesById;
+  }
+  
+  @VisibleForTesting
+  public GSet<CachedBlock, CachedBlock> getCachedBlocks() {
+    assert namesystem.hasReadOrWriteLock();
+    return cachedBlocks;
+  }
+
+  private long getNextEntryId() throws IOException {
+    assert namesystem.hasWriteLock();
+    if (nextEntryId == Long.MAX_VALUE) {
+      throw new IOException("No more available IDs");
+    }
+    return nextEntryId++;
+  }
+
+  private PathBasedCacheEntry findEntry(PathBasedCacheDirective directive) {
+    assert namesystem.hasReadOrWriteLock();
     List<PathBasedCacheEntry> existing =
         entriesByPath.get(directive.getPath().toUri().getPath());
     if (existing == null) {
@@ -151,56 +281,10 @@ public final class CacheManager {
     return null;
   }
 
-  /**
-   * Add a new PathBasedCacheEntry, skipping any validation checks. Called
-   * directly when reloading CacheManager state from FSImage.
-   * 
-   * @throws IOException if unable to cache the entry
-   */
-  private void unprotectedAddEntry(PathBasedCacheEntry entry)
-      throws IOException {
-    assert namesystem.hasWriteLock();
-    // Add it to the various maps
-    entriesById.put(entry.getEntryId(), entry);
-    String path = entry.getPath();
-    List<PathBasedCacheEntry> entryList = entriesByPath.get(path);
-    if (entryList == null) {
-      entryList = new ArrayList<PathBasedCacheEntry>(1);
-      entriesByPath.put(path, entryList);
-    }
-    entryList.add(entry);
-    // Set the path as cached in the namesystem
-    try {
-      INode node = dir.getINode(entry.getPath());
-      if (node != null && node.isFile()) {
-        INodeFile file = node.asFile();
-        // TODO: adjustable cache replication factor
-        namesystem.setCacheReplicationInt(entry.getPath(),
-            file.getBlockReplication());
-      } else {
-        LOG.warn("Path " + entry.getPath() + " is not a file");
-      }
-    } catch (IOException ioe) {
-      LOG.info("unprotectedAddEntry " + entry +": failed to cache file: " +
-          ioe.getClass().getName() +": " + ioe.getMessage());
-      throw ioe;
-    }
-  }
-
-  /**
-   * Add a new PathBasedCacheDirective if valid, returning a corresponding
-   * PathBasedCacheDescriptor to the user.
-   * 
-   * @param directive Directive describing the cache entry being added
-   * @param pc Permission checker used to validate that the calling user has
-   *          access to the destination cache pool
-   * @return Corresponding PathBasedCacheDescriptor for the new cache entry
-   * @throws IOException if the directive is invalid or was otherwise
-   *           unsuccessful
-   */
-  public synchronized PathBasedCacheDescriptor addDirective(
+  public PathBasedCacheDescriptor addDirective(
       PathBasedCacheDirective directive, FSPermissionChecker pc)
       throws IOException {
+    assert namesystem.hasWriteLock();
     CachePool pool = cachePools.get(directive.getPool());
     if (pool == null) {
       LOG.info("addDirective " + directive + ": pool not found.");
@@ -225,47 +309,37 @@ public final class CacheManager {
           "existing directive " + existing + " in this pool.");
       return existing.getDescriptor();
     }
-
-    // Success!
-    PathBasedCacheDescriptor d = unprotectedAddDirective(directive);
-    LOG.info("addDirective " + directive + ": added cache directive "
-        + directive);
-    return d;
-  }
-
-  /**
-   * Assigns a new entry ID to a validated PathBasedCacheDirective and adds
-   * it to the CacheManager. Called directly when replaying the edit log.
-   * 
-   * @param directive Directive being added
-   * @return PathBasedCacheDescriptor for the directive
-   * @throws IOException
-   */
-  PathBasedCacheDescriptor unprotectedAddDirective(
-      PathBasedCacheDirective directive) throws IOException {
-    assert namesystem.hasWriteLock();
-    CachePool pool = cachePools.get(directive.getPool());
     // Add a new entry with the next available ID.
     PathBasedCacheEntry entry;
-    entry = new PathBasedCacheEntry(getNextEntryId(),
-        directive.getPath().toUri().getPath(),
-        directive.getReplication(), pool);
-
-    unprotectedAddEntry(entry);
+    try {
+      entry = new PathBasedCacheEntry(getNextEntryId(),
+          directive.getPath().toUri().getPath(),
+          directive.getReplication(), pool);
+    } catch (IOException ioe) {
+      throw new UnexpectedAddPathBasedCacheDirectiveException(directive);
+    }
+    LOG.info("addDirective " + directive + ": added cache directive "
+        + directive);
 
+    // Success!
+    // First, add it to the various maps
+    entriesById.put(entry.getEntryId(), entry);
+    String path = directive.getPath().toUri().getPath();
+    List<PathBasedCacheEntry> entryList = entriesByPath.get(path);
+    if (entryList == null) {
+      entryList = new ArrayList<PathBasedCacheEntry>(1);
+      entriesByPath.put(path, entryList);
+    }
+    entryList.add(entry);
+    if (monitor != null) {
+      monitor.kick();
+    }
     return entry.getDescriptor();
   }
 
-  /**
-   * Remove the PathBasedCacheEntry corresponding to a descriptor ID from
-   * the CacheManager.
-   * 
-   * @param id of the PathBasedCacheDescriptor
-   * @param pc Permissions checker used to validated the request
-   * @throws IOException
-   */
-  public synchronized void removeDescriptor(long id, FSPermissionChecker pc)
+  public void removeDescriptor(long id, FSPermissionChecker pc)
       throws IOException {
+    assert namesystem.hasWriteLock();
     // Check for invalid IDs.
     if (id <= 0) {
       LOG.info("removeDescriptor " + id + ": invalid non-positive " +
@@ -290,20 +364,6 @@ public final class CacheManager {
       throw new RemovePermissionDeniedException(id);
     }
     
-    unprotectedRemoveDescriptor(id);
-  }
-
-  /**
-   * Unchecked internal method used to remove a PathBasedCacheEntry from the
-   * CacheManager. Called directly when replaying the edit log.
-   * 
-   * @param id of the PathBasedCacheDescriptor corresponding to the entry that
-   *          is being removed
-   * @throws IOException
-   */
-  void unprotectedRemoveDescriptor(long id) throws IOException {
-    assert namesystem.hasWriteLock();
-    PathBasedCacheEntry existing = entriesById.get(id);
     // Remove the corresponding entry in entriesByPath.
     String path = existing.getDescriptor().getPath().toUri().getPath();
     List<PathBasedCacheEntry> entries = entriesByPath.get(path);
@@ -314,26 +374,16 @@ public final class CacheManager {
       entriesByPath.remove(path);
     }
     entriesById.remove(id);
-
-    // Set the path as uncached in the namesystem
-    try {
-      INode node = dir.getINode(existing.getDescriptor().getPath().toUri().
-          getPath());
-      if (node != null && node.isFile()) {
-        namesystem.setCacheReplicationInt(existing.getDescriptor().getPath().
-            toUri().getPath(), (short) 0);
-      }
-    } catch (IOException e) {
-      LOG.warn("removeDescriptor " + id + ": failure while setting cache"
-          + " replication factor", e);
-      throw e;
+    if (monitor != null) {
+      monitor.kick();
     }
     LOG.info("removeDescriptor successful for PathCacheEntry id " + id);
   }
 
-  public synchronized BatchedListEntries<PathBasedCacheDescriptor> 
+  public BatchedListEntries<PathBasedCacheDescriptor> 
         listPathBasedCacheDescriptors(long prevId, String filterPool,
             String filterPath, FSPermissionChecker pc) throws IOException {
+    assert namesystem.hasReadOrWriteLock();
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     if (filterPath != null) {
       if (!DFSUtil.isValidName(filterPath)) {
@@ -370,12 +420,13 @@ public final class CacheManager {
    * 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 the created CachePool
+   *
+   * @param info    The info for the cache pool to create.
+   * @return        Information about the cache pool we created.
    */
-  public synchronized CachePoolInfo addCachePool(CachePoolInfo info)
+  public CachePoolInfo addCachePool(CachePoolInfo info)
       throws IOException {
+    assert namesystem.hasWriteLock();
     CachePoolInfo.validate(info);
     String poolName = info.getPoolName();
     CachePool pool = cachePools.get(poolName);
@@ -384,20 +435,8 @@ public final class CacheManager {
     }
     pool = CachePool.createFromInfoAndDefaults(info);
     cachePools.put(pool.getPoolName(), pool);
-    return pool.getInfo(true);
-  }
-
-  /**
-   * Internal unchecked method used to add a CachePool. Called directly when
-   * reloading CacheManager state from the FSImage or edit log.
-   * 
-   * @param pool to be added
-   */
-  void unprotectedAddCachePool(CachePoolInfo info) {
-    assert namesystem.hasWriteLock();
-    CachePool pool = CachePool.createFromInfo(info);
-    cachePools.put(pool.getPoolName(), pool);
     LOG.info("created new cache pool " + pool);
+    return pool.getInfo(true);
   }
 
   /**
@@ -408,8 +447,9 @@ public final class CacheManager {
    * @param info
    *          The info for the cache pool to modify.
    */
-  public synchronized void modifyCachePool(CachePoolInfo info)
+  public void modifyCachePool(CachePoolInfo info)
       throws IOException {
+    assert namesystem.hasWriteLock();
     CachePoolInfo.validate(info);
     String poolName = info.getPoolName();
     CachePool pool = cachePools.get(poolName);
@@ -455,8 +495,9 @@ public final class CacheManager {
    * @param poolName
    *          The name for the cache pool to remove.
    */
-  public synchronized void removeCachePool(String poolName)
+  public void removeCachePool(String poolName)
       throws IOException {
+    assert namesystem.hasWriteLock();
     CachePoolInfo.validateName(poolName);
     CachePool pool = cachePools.remove(poolName);
     if (pool == null) {
@@ -475,10 +516,14 @@ public final class CacheManager {
         iter.remove();
       }
     }
+    if (monitor != null) {
+      monitor.kick();
+    }
   }
 
-  public synchronized BatchedListEntries<CachePoolInfo>
+  public BatchedListEntries<CachePoolInfo>
       listCachePools(FSPermissionChecker pc, String prevKey) {
+    assert namesystem.hasReadOrWriteLock();
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     ArrayList<CachePoolInfo> results = 
         new ArrayList<CachePoolInfo>(NUM_PRE_ALLOCATED_ENTRIES);
@@ -497,9 +542,104 @@ public final class CacheManager {
     return new BatchedListEntries<CachePoolInfo>(results, false);
   }
 
-  /*
-   * FSImage related serialization and deserialization code
-   */
+  public void setCachedLocations(LocatedBlock block) {
+    if (!enabled) {
+      return;
+    }
+    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 BlockListAsLongs report) throws IOException {
+    if (!enabled) {
+      LOG.info("Ignoring cache report from " + datanodeID +
+          " because " + DFS_NAMENODE_CACHING_ENABLED_KEY + " = false. " +
+          "number of blocks: " + report.getNumberOfBlocks());
+      return;
+    }
+    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, report);
+    } 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: " + report.getNumberOfBlocks()
+        + ", processing time: " + (endTime - startTime) + " msecs");
+  }
+
+  private void processCacheReportImpl(final DatanodeDescriptor datanode,
+      final BlockListAsLongs report) {
+    CachedBlocksList cached = datanode.getCached();
+    cached.clear();
+    BlockReportIterator itBR = report.getBlockReportIterator();
+    while (itBR.hasNext()) {
+      Block block = itBR.next();
+      ReplicaState iState = itBR.getCurrentReplicaState();
+      if (iState != ReplicaState.FINALIZED) {
+        LOG.error("Cached block report contained unfinalized block " + block);
+        continue;
+      }
+      BlockInfo blockInfo = blockManager.getStoredBlock(block);
+      if (blockInfo.getGenerationStamp() < block.getGenerationStamp()) {
+        // The NameNode will eventually remove or update the out-of-date block.
+        // Until then, we pretend that it isn't cached.
+        LOG.warn("Genstamp in cache report disagrees with our genstamp for " +
+          block + ": expected genstamp " + blockInfo.getGenerationStamp());
+        continue;
+      }
+      Collection<DatanodeDescriptor> corruptReplicas =
+          blockManager.getCorruptReplicas(blockInfo);
+      if ((corruptReplicas != null) && corruptReplicas.contains(datanode)) {
+        // The NameNode will eventually remove or update the corrupt block.
+        // Until then, we pretend that it isn't cached.
+        LOG.warn("Ignoring cached replica on " + datanode + " of " + block +
+            " because it is corrupt.");
+        continue;
+      }
+      CachedBlock cachedBlock =
+          new CachedBlock(block.getBlockId(), (short)0, false);
+      CachedBlock prevCachedBlock = cachedBlocks.get(cachedBlock);
+      // Use the existing CachedBlock if it's present; otherwise,
+      // insert a new one.
+      if (prevCachedBlock != null) {
+        cachedBlock = prevCachedBlock;
+      } else {
+        cachedBlocks.put(cachedBlock);
+      }
+      if (!cachedBlock.isPresent(datanode.getCached())) {
+        datanode.getCached().add(cachedBlock);
+      }
+      if (cachedBlock.isPresent(datanode.getPendingCached())) {
+        datanode.getPendingCached().remove(cachedBlock);
+      }
+    }
+  }
 
   /**
    * Saves the current state of the CacheManager to the DataOutput. Used
@@ -508,7 +648,7 @@ public final class CacheManager {
    * @param sdPath path of the storage directory
    * @throws IOException
    */
-  public synchronized void saveState(DataOutput out, String sdPath)
+  public void saveState(DataOutput out, String sdPath)
       throws IOException {
     out.writeLong(nextEntryId);
     savePools(out, sdPath);
@@ -521,7 +661,8 @@ public final class CacheManager {
    * @param in DataInput from which to restore state
    * @throws IOException
    */
-  public synchronized void loadState(DataInput in) throws IOException {
+  public void loadState(DataInput in) throws IOException {
+    assert namesystem.hasWriteLock();
     nextEntryId = in.readLong();
     // pools need to be loaded first since entries point to their parent pool
     loadPools(in);
@@ -531,7 +672,7 @@ public final class CacheManager {
   /**
    * Save cache pools to fsimage
    */
-  private synchronized void savePools(DataOutput out,
+  private void savePools(DataOutput out,
       String sdPath) throws IOException {
     StartupProgress prog = NameNode.getStartupProgress();
     Step step = new Step(StepType.CACHE_POOLS, sdPath);
@@ -549,7 +690,7 @@ public final class CacheManager {
   /*
    * Save cache entries to fsimage
    */
-  private synchronized void saveEntries(DataOutput out, String sdPath)
+  private void saveEntries(DataOutput out, String sdPath)
       throws IOException {
     StartupProgress prog = NameNode.getStartupProgress();
     Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
@@ -560,6 +701,7 @@ public final class CacheManager {
     for (PathBasedCacheEntry entry: entriesById.values()) {
       out.writeLong(entry.getEntryId());
       Text.writeString(out, entry.getPath());
+      out.writeShort(entry.getReplication());
       Text.writeString(out, entry.getPool().getPoolName());
       counter.increment();
     }
@@ -569,7 +711,7 @@ public final class CacheManager {
   /**
    * Load cache pools from fsimage
    */
-  private synchronized void loadPools(DataInput in)
+  private void loadPools(DataInput in)
       throws IOException {
     StartupProgress prog = NameNode.getStartupProgress();
     Step step = new Step(StepType.CACHE_POOLS);
@@ -578,8 +720,7 @@ public final class CacheManager {
     prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfPools);
     Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
     for (int i = 0; i < numberOfPools; i++) {
-      CachePoolInfo info = CachePoolInfo.readFrom(in);
-      unprotectedAddCachePool(info);
+      addCachePool(CachePoolInfo.readFrom(in));
       counter.increment();
     }
     prog.endStep(Phase.LOADING_FSIMAGE, step);
@@ -588,7 +729,7 @@ public final class CacheManager {
   /**
    * Load cache entries from the fsimage
    */
-  private synchronized void loadEntries(DataInput in) throws IOException {
+  private void loadEntries(DataInput in) throws IOException {
     StartupProgress prog = NameNode.getStartupProgress();
     Step step = new Step(StepType.CACHE_ENTRIES);
     prog.beginStep(Phase.LOADING_FSIMAGE, step);
@@ -602,12 +743,24 @@ public final class CacheManager {
       String poolName = Text.readString(in);
       // Get pool reference by looking it up in the map
       CachePool pool = cachePools.get(poolName);
+      if (pool != null) {
+        throw new IOException("Entry refers to pool " + poolName +
+            ", which does not exist.");
+      }
       PathBasedCacheEntry entry =
-        new PathBasedCacheEntry(entryId, path, replication, pool);
-      unprotectedAddEntry(entry);
+          new PathBasedCacheEntry(entryId, path, replication, pool);
+      if (entriesById.put(entry.getEntryId(), entry) != null) {
+        throw new IOException("An entry with ID " + entry.getEntryId() +
+            " already exists");
+      }
+      List<PathBasedCacheEntry> entries = entriesByPath.get(entry.getPath());
+      if (entries == null) {
+        entries = new LinkedList<PathBasedCacheEntry>();
+        entriesByPath.put(entry.getPath(), entries);
+      }
+      entries.add(entry);
       counter.increment();
     }
     prog.endStep(Phase.LOADING_FSIMAGE, step);
   }
-
 }

+ 14 - 52
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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
@@ -1092,52 +1093,6 @@ public class FSDirectory implements Closeable {
     return file.getBlocks();
   }
 
-  /**
-   * Set cache replication for a file
-   * 
-   * @param src file name
-   * @param replication new replication
-   * @param blockRepls block replications - output parameter
-   * @return array of file blocks
-   * @throws QuotaExceededException
-   * @throws SnapshotAccessControlException
-   */
-  Block[] setCacheReplication(String src, short replication, short[] blockRepls)
-      throws QuotaExceededException, UnresolvedLinkException,
-      SnapshotAccessControlException {
-    waitForReady();
-    writeLock();
-    try {
-      return unprotectedSetCacheReplication(src, replication, blockRepls);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  Block[] unprotectedSetCacheReplication(String src, short replication,
-      short[] blockRepls) throws QuotaExceededException,
-      UnresolvedLinkException, SnapshotAccessControlException {
-    assert hasWriteLock();
-
-    final INodesInPath iip = rootDir.getINodesInPath4Write(src, true);
-    final INode inode = iip.getLastINode();
-    if (inode == null || !inode.isFile()) {
-      return null;
-    }
-    INodeFile file = inode.asFile();
-    final short oldBR = file.getCacheReplication();
-
-    // TODO: Update quotas here as repl goes up or down
-    file.setCacheReplication(replication);
-    final short newBR = file.getCacheReplication();
-
-    if (blockRepls != null) {
-      blockRepls[0] = oldBR;
-      blockRepls[1] = newBR;
-    }
-    return file.getBlocks();
-  }
-
   /**
    * @param path the file path
    * @return the block size of the file. 
@@ -2638,12 +2593,19 @@ public class FSDirectory implements Closeable {
     int childrenNum = node.isDirectory() ? 
         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.
+    CacheManager cacheManager = namesystem.getCacheManager();
+    for (LocatedBlock lb: loc.getLocatedBlocks()) {
+      cacheManager.setCachedLocations(lb);
+    }
+    return status;
   }
 
     

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

@@ -648,8 +648,7 @@ public class FSEditLogLoader {
           setPool(addOp.pool).
           build();
       PathBasedCacheDescriptor descriptor =
-          fsNamesys.getCacheManager().unprotectedAddDirective(d);
-
+          fsNamesys.getCacheManager().addDirective(d, null);
       if (toAddRetryCache) {
         fsNamesys.addCacheEntryWithPayload(op.rpcClientId, op.rpcCallId,
             descriptor);
@@ -659,8 +658,7 @@ public class FSEditLogLoader {
     case OP_REMOVE_PATH_BASED_CACHE_DESCRIPTOR: {
       RemovePathBasedCacheDescriptorOp removeOp =
           (RemovePathBasedCacheDescriptorOp) op;
-      fsNamesys.getCacheManager().unprotectedRemoveDescriptor(removeOp.id);
-
+      fsNamesys.getCacheManager().removeDescriptor(removeOp.id, null);
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
       }
@@ -668,8 +666,7 @@ public class FSEditLogLoader {
     }
     case OP_ADD_CACHE_POOL: {
       AddCachePoolOp addOp = (AddCachePoolOp) op;
-      fsNamesys.getCacheManager().unprotectedAddCachePool(addOp.info);
-
+      fsNamesys.getCacheManager().addCachePool(addOp.info);
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
       }
@@ -678,7 +675,6 @@ public class FSEditLogLoader {
     case OP_MODIFY_CACHE_POOL: {
       ModifyCachePoolOp modifyOp = (ModifyCachePoolOp) op;
       fsNamesys.getCacheManager().modifyCachePool(modifyOp.info);
-
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
       }
@@ -687,7 +683,6 @@ public class FSEditLogLoader {
     case OP_REMOVE_CACHE_POOL: {
       RemoveCachePoolOp removeOp = (RemoveCachePoolOp) op;
       fsNamesys.getCacheManager().removeCachePool(removeOp.poolName);
-
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
       }

+ 18 - 53
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -374,7 +374,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private final BlockManager blockManager;
   private final SnapshotManager snapshotManager;
   private final CacheManager cacheManager;
-  private final CacheReplicationManager cacheReplicationManager;
   private final DatanodeStatistics datanodeStatistics;
 
   // Block pool ID used by this namenode
@@ -702,9 +701,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       this.dtSecretManager = createDelegationTokenSecretManager(conf);
       this.dir = new FSDirectory(fsImage, this, conf);
       this.snapshotManager = new SnapshotManager(dir);
-      this.cacheManager = new CacheManager(this, dir, conf);
-      this.cacheReplicationManager = new CacheReplicationManager(this,
-          blockManager, blockManager.getDatanodeManager(), this, conf);
+      writeLock();
+      try {
+        this.cacheManager = new CacheManager(this, conf, blockManager);
+      } finally {
+        writeUnlock();
+      }
       this.safeMode = new SafeModeInfo(conf);
       this.auditLoggers = initAuditLoggers(conf);
       this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
@@ -881,7 +883,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         getCompleteBlocksTotal());
       setBlockTotal();
       blockManager.activate(conf);
-      cacheReplicationManager.activate();
     } finally {
       writeUnlock();
     }
@@ -898,7 +899,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     try {
       if (blockManager != null) blockManager.close();
-      if (cacheReplicationManager != null) cacheReplicationManager.close();
+      cacheManager.deactivate();
     } finally {
       writeUnlock();
     }
@@ -930,8 +931,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         blockManager.clearQueues();
         blockManager.processAllPendingDNMessages();
 
-        cacheReplicationManager.clearQueues();
-
         if (!isInSafeMode() ||
             (isInSafeMode() && safeMode.isPopulatingReplQueues())) {
           LOG.info("Reprocessing replication and invalidation queues");
@@ -964,6 +963,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       //ResourceMonitor required only at ActiveNN. See HDFS-2914
       this.nnrmthread = new Daemon(new NameNodeResourceMonitor());
       nnrmthread.start();
+      cacheManager.activate();
+      blockManager.getDatanodeManager().setSendCachingCommands(true);
     } finally {
       writeUnlock();
     }
@@ -998,6 +999,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         // so that the tailer starts from the right spot.
         dir.fsImage.updateLastAppliedTxIdFromWritten();
       }
+      cacheManager.deactivate();
+      blockManager.getDatanodeManager().setSendCachingCommands(false);
     } finally {
       writeUnlock();
     }
@@ -1442,10 +1445,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         blockManager.getDatanodeManager().sortLocatedBlocks(
                               clientMachine, lastBlockList);
       }
-      // Set caching information for the block list
-      for (LocatedBlock lb: blocks.getLocatedBlocks()) {
-        cacheReplicationManager.setCachedLocations(lb);
-      }
     }
     return blocks;
   }
@@ -1553,8 +1552,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           length = Math.min(length, fileSize - offset);
           isUc = false;
         }
-        return blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
+        LocatedBlocks blocks =
+          blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
             isUc, offset, length, needBlockToken, iip.isSnapshot());
+        // Set caching information for the located blocks.
+        for (LocatedBlock lb: blocks.getLocatedBlocks()) {
+          cacheManager.setCachedLocations(lb);
+        }
+        return blocks;
       } finally {
         if (isReadOp) {
           readUnlock();
@@ -1928,42 +1933,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return isFile;
   }
 
-  boolean setCacheReplicationInt(String src, final short replication)
-      throws IOException {
-    final boolean isFile;
-    FSPermissionChecker pc = getPermissionChecker();
-    checkOperation(OperationCategory.WRITE);
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
-    writeLock();
-    try {
-      checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot set replication for " + src, safeMode);
-      }
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
-      if (isPermissionEnabled) {
-        checkPathAccess(pc, src, FsAction.WRITE);
-      }
-
-      final short[] blockRepls = new short[2]; // 0: old, 1: new
-      final Block[] blocks = dir.setCacheReplication(src, replication,
-          blockRepls);
-      isFile = (blocks != null);
-      if (isFile) {
-        cacheReplicationManager.setCacheReplication(blockRepls[0],
-            blockRepls[1], src, blocks);
-      }
-    } finally {
-      writeUnlock();
-    }
-
-    getEditLog().logSync();
-    if (isFile) {
-      logAuditEvent(true, "setCacheReplication", src);
-    }
-    return isFile;
-  }
-
   long getPreferredBlockSize(String filename) 
       throws IOException, UnresolvedLinkException {
     FSPermissionChecker pc = getPermissionChecker();
@@ -6506,10 +6475,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public CacheManager getCacheManager() {
     return cacheManager;
   }
-  /** @return the cache replication manager. */
-  public CacheReplicationManager getCacheReplicationManager() {
-    return cacheReplicationManager;
-  }
 
   @Override  // NameNodeMXBean
   public String getCorruptFiles() {

+ 0 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -104,8 +104,6 @@ public class INodeFile extends INodeWithAdditionalFields
 
   private BlockInfo[] blocks;
 
-  private short cacheReplication = 0;
-
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, long atime,
       BlockInfo[] blklist, short replication, long preferredBlockSize) {
     super(id, name, permissions, mtime, atime);
@@ -201,18 +199,6 @@ public class INodeFile extends INodeWithAdditionalFields
     return nodeToUpdate;
   }
 
-  @Override
-  public void setCacheReplication(short cacheReplication) {
-    Preconditions.checkArgument(cacheReplication <= getBlockReplication(),
-        "Cannot set cache replication higher than block replication factor");
-    this.cacheReplication = cacheReplication;
-  }
-
-  @Override
-  public short getCacheReplication() {
-    return cacheReplication;
-  }
-
   /** @return preferred block size (in bytes) of the file. */
   @Override
   public long getPreferredBlockSize() {

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

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

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

@@ -997,9 +997,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
            + "from " + nodeReg + " " + blist.getNumberOfBlocks()
            + " blocks");
     }
-
-    namesystem.getCacheReplicationManager()
-        .processCacheReport(nodeReg, poolId, blist);
+    namesystem.getCacheManager().processCacheReport(nodeReg, blist);
     return null;
   }
 

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

@@ -1470,6 +1470,18 @@
   </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.datanode.fsdatasetcache.max.threads.per.volume</name>
   <value>4</value>

+ 0 - 256
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCacheReplicationManager.java

@@ -1,256 +0,0 @@
-/**
- * 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.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystemTestHelper;
-import org.apache.hadoop.fs.HdfsBlockLocation;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.io.nativeio.NativeIO;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestCacheReplicationManager {
-
-  private static final long BLOCK_SIZE = 512;
-  private static final int REPL_FACTOR = 3;
-  private static final int NUM_DATANODES = 4;
-  // Most Linux installs allow a default of 64KB locked memory
-  private static final long CACHE_CAPACITY = 64 * 1024 / NUM_DATANODES;
-
-  private static Configuration conf;
-  private static MiniDFSCluster cluster = null;
-  private static DistributedFileSystem dfs;
-  private static NameNode nn;
-  private static NamenodeProtocols nnRpc;
-  private static CacheReplicationManager cacheReplManager;
-  final private static FileSystemTestHelper helper = new FileSystemTestHelper();
-  private static Path rootDir;
-
-  @Before
-  public void setUp() throws Exception {
-
-    assumeTrue(NativeIO.isAvailable());
-
-    conf = new HdfsConfiguration();
-    conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
-        CACHE_CAPACITY);
-    conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
-    conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, true);
-    conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000);
-
-    cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(NUM_DATANODES).build();
-    cluster.waitActive();
-
-    dfs = cluster.getFileSystem();
-    nn = cluster.getNameNode();
-    nnRpc = nn.getRpcServer();
-    cacheReplManager = nn.getNamesystem().getCacheReplicationManager();
-    rootDir = helper.getDefaultWorkingDirectory(dfs);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (dfs != null) {
-      dfs.close();
-    }
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  private int countNumCachedBlocks() {
-    return cacheReplManager.cachedBlocksMap.size();
-  }
-
-  private void waitForExpectedNumCachedBlocks(final int expected)
-      throws Exception {
-    int actual = countNumCachedBlocks();
-    while (expected != actual)  {
-      Thread.sleep(500);
-      actual = countNumCachedBlocks();
-    }
-    waitForExpectedNumCachedReplicas(expected*REPL_FACTOR);
-  }
-
-  private void waitForExpectedNumCachedReplicas(final int expected)
-      throws Exception {
-    BlocksMap cachedBlocksMap = cacheReplManager.cachedBlocksMap;
-    int actual = 0;
-    while (expected != actual) {
-      Thread.sleep(500);
-      nn.getNamesystem().readLock();
-      try {
-        actual = 0;
-        for (BlockInfo b : cachedBlocksMap.getBlocks()) {
-          actual += cachedBlocksMap.numNodes(b);
-        }
-      } finally {
-        nn.getNamesystem().readUnlock();
-      }
-    }
-  }
-
-  @Test(timeout=60000)
-  public void testCachePaths() throws Exception {
-    // Create the pool
-    final String pool = "friendlyPool";
-    nnRpc.addCachePool(new CachePoolInfo("friendlyPool"));
-    // Create some test files
-    final int numFiles = 2;
-    final int numBlocksPerFile = 2;
-    final List<String> paths = new ArrayList<String>(numFiles);
-    for (int i=0; i<numFiles; i++) {
-      Path p = new Path(rootDir, "testCachePaths-" + i);
-      FileSystemTestHelper.createFile(dfs, p, numBlocksPerFile, (int)BLOCK_SIZE);
-      paths.add(p.toUri().getPath());
-    }
-    // Check the initial statistics at the namenode
-    int expected = 0;
-    waitForExpectedNumCachedBlocks(expected);
-    // Cache and check each path in sequence
-    for (int i=0; i<numFiles; i++) {
-      PathBasedCacheDirective directive = new PathBasedCacheDirective.Builder().
-          setPath(new Path(paths.get(i))).
-          setPool(pool).
-          build();
-      PathBasedCacheDescriptor descriptor =
-          nnRpc.addPathBasedCacheDirective(directive);
-      assertEquals("Descriptor does not match requested path", paths.get(i),
-          descriptor.getPath().toUri().getPath());
-      assertEquals("Descriptor does not match requested pool", pool,
-          descriptor.getPool());
-      expected += numBlocksPerFile;
-      waitForExpectedNumCachedBlocks(expected);
-      HdfsBlockLocation[] locations =
-          (HdfsBlockLocation[]) dfs.getFileBlockLocations(
-              new Path(paths.get(i)), 0, numBlocksPerFile * BLOCK_SIZE);
-      assertEquals("Unexpected number of locations", numBlocksPerFile,
-          locations.length);
-      for (HdfsBlockLocation loc: locations) {
-        assertEquals("Block should be present on all datanodes",
-            3, loc.getHosts().length);
-        DatanodeInfo[] cachedLocs = loc.getLocatedBlock().getCachedLocations();
-        assertEquals("Block should be cached on all datanodes",
-            loc.getHosts().length, cachedLocs.length);
-      }
-    }
-    // Uncache and check each path in sequence
-    RemoteIterator<PathBasedCacheDescriptor> entries =
-        nnRpc.listPathBasedCacheDescriptors(0, null, null);
-    for (int i=0; i<numFiles; i++) {
-      PathBasedCacheDescriptor descriptor = entries.next();
-      nnRpc.removePathBasedCacheDescriptor(descriptor.getEntryId());
-      expected -= numBlocksPerFile;
-      waitForExpectedNumCachedBlocks(expected);
-    }
-  }
-
-  @Test(timeout=60000)
-  public void testCacheManagerRestart() throws Exception {
-    // Create and validate a pool
-    final String pool = "poolparty";
-    String groupName = "partygroup";
-    FsPermission mode = new FsPermission((short)0777);
-    int weight = 747;
-    dfs.addCachePool(new CachePoolInfo(pool)
-        .setGroupName(groupName)
-        .setMode(mode)
-        .setWeight(weight));
-    RemoteIterator<CachePoolInfo> pit = dfs.listCachePools();
-    assertTrue("No cache pools found", pit.hasNext());
-    CachePoolInfo info = pit.next();
-    assertEquals(pool, info.getPoolName());
-    assertEquals(groupName, info.getGroupName());
-    assertEquals(mode, info.getMode());
-    assertEquals(weight, (int)info.getWeight());
-    assertFalse("Unexpected # of cache pools found", pit.hasNext());
-
-    // Create some cache entries
-    int numEntries = 10;
-    String entryPrefix = "/party-";
-    for (int i=0; i<numEntries; i++) {
-      dfs.addPathBasedCacheDirective(new PathBasedCacheDirective.Builder().
-          setPath(new Path(entryPrefix + i)).
-          setPool(pool).
-          build());
-    }
-    RemoteIterator<PathBasedCacheDescriptor> dit
-        = dfs.listPathBasedCacheDescriptors(null, null);
-    for (int i=0; i<numEntries; i++) {
-      assertTrue("Unexpected # of cache entries: " + i, dit.hasNext());
-      PathBasedCacheDescriptor cd = dit.next();
-      assertEquals(i+1, cd.getEntryId());
-      assertEquals(entryPrefix + i, cd.getPath().toUri().getPath());
-      assertEquals(pool, cd.getPool());
-    }
-    assertFalse("Unexpected # of cache descriptors found", dit.hasNext());
-
-    // Restart namenode
-    cluster.restartNameNode();
-
-    // Check that state came back up
-    pit = dfs.listCachePools();
-    assertTrue("No cache pools found", pit.hasNext());
-    info = pit.next();
-    assertEquals(pool, info.getPoolName());
-    assertEquals(pool, info.getPoolName());
-    assertEquals(groupName, info.getGroupName());
-    assertEquals(mode, info.getMode());
-    assertEquals(weight, (int)info.getWeight());
-    assertFalse("Unexpected # of cache pools found", pit.hasNext());
-
-    dit = dfs.listPathBasedCacheDescriptors(null, null);
-    for (int i=0; i<numEntries; i++) {
-      assertTrue("Unexpected # of cache entries: " + i, dit.hasNext());
-      PathBasedCacheDescriptor cd = dit.next();
-      assertEquals(i+1, cd.getEntryId());
-      assertEquals(entryPrefix + i, cd.getPath().toUri().getPath());
-      assertEquals(pool, cd.getPool());
-    }
-    assertFalse("Unexpected # of cache descriptors found", dit.hasNext());
-  }
-
-}

+ 151 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCachedBlocksList.java

@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
+import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCachedBlocksList {
+  public static final Log LOG = LogFactory.getLog(TestCachedBlocksList.class);
+
+  @Test(timeout=60000)
+  public void testSingleList() {
+    DatanodeDescriptor dn = new DatanodeDescriptor(
+      new DatanodeID("127.0.0.1", "localhost", "abcd", 5000, 5001, 5002));
+    CachedBlock[] blocks = new CachedBlock[] {
+          new CachedBlock(0L, (short)1, true),
+          new CachedBlock(1L, (short)1, true),
+          new CachedBlock(2L, (short)1, true),
+      };
+    // check that lists are empty
+    Assert.assertTrue("expected pending cached list to start off empty.", 
+        !dn.getPendingCached().iterator().hasNext());
+    Assert.assertTrue("expected cached list to start off empty.", 
+        !dn.getCached().iterator().hasNext());
+    Assert.assertTrue("expected pending uncached list to start off empty.", 
+        !dn.getPendingUncached().iterator().hasNext());
+    // add a block to the back
+    Assert.assertTrue(dn.getCached().add(blocks[0]));
+    Assert.assertTrue("expected pending cached list to still be empty.", 
+        !dn.getPendingCached().iterator().hasNext());
+    Assert.assertEquals("failed to insert blocks[0]", blocks[0],
+        dn.getCached().iterator().next());
+    Assert.assertTrue("expected pending uncached list to still be empty.", 
+        !dn.getPendingUncached().iterator().hasNext());
+    // add another block to the back
+    Assert.assertTrue(dn.getCached().add(blocks[1]));
+    Iterator<CachedBlock> iter = dn.getCached().iterator();
+    Assert.assertEquals(blocks[0], iter.next());
+    Assert.assertEquals(blocks[1], iter.next());
+    Assert.assertTrue(!iter.hasNext());
+    // add a block to the front
+    Assert.assertTrue(dn.getCached().addFirst(blocks[2]));
+    iter = dn.getCached().iterator();
+    Assert.assertEquals(blocks[2], iter.next());
+    Assert.assertEquals(blocks[0], iter.next());
+    Assert.assertEquals(blocks[1], iter.next());
+    Assert.assertTrue(!iter.hasNext());
+    // remove a block from the middle
+    Assert.assertTrue(dn.getCached().remove(blocks[0]));
+    iter = dn.getCached().iterator();
+    Assert.assertEquals(blocks[2], iter.next());
+    Assert.assertEquals(blocks[1], iter.next());
+    Assert.assertTrue(!iter.hasNext());
+    // remove all blocks
+    dn.getCached().clear();
+    Assert.assertTrue("expected cached list to be empty after clear.", 
+        !dn.getPendingCached().iterator().hasNext());
+  }
+
+  private void testAddElementsToList(CachedBlocksList list,
+      CachedBlock[] blocks) {
+    Assert.assertTrue("expected list to start off empty.", 
+        !list.iterator().hasNext());
+    for (CachedBlock block : blocks) {
+      Assert.assertTrue(list.add(block));
+    }
+  }
+
+  private void testRemoveElementsFromList(Random r,
+      CachedBlocksList list, CachedBlock[] blocks) {
+    int i = 0;
+    for (Iterator<CachedBlock> iter = list.iterator(); iter.hasNext(); ) {
+      Assert.assertEquals(blocks[i], iter.next());
+      i++;
+    }
+    if (r.nextBoolean()) {
+      LOG.info("Removing via iterator");
+      for (Iterator<CachedBlock> iter = list.iterator(); iter.hasNext() ;) {
+        iter.next();
+        iter.remove();
+      }
+    } else {
+      LOG.info("Removing in pseudo-random order");
+      CachedBlock[] remainingBlocks = Arrays.copyOf(blocks, blocks.length);
+      for (int removed = 0; removed < remainingBlocks.length; ) {
+        int toRemove = r.nextInt(remainingBlocks.length);
+        if (remainingBlocks[toRemove] != null) {
+          Assert.assertTrue(list.remove(remainingBlocks[toRemove]));
+          remainingBlocks[toRemove] = null;
+          removed++;
+        }
+      }
+    }
+    Assert.assertTrue("expected list to be empty after everything " +
+        "was removed.", !list.iterator().hasNext());
+  }
+
+  @Test(timeout=60000)
+  public void testMultipleLists() {
+    DatanodeDescriptor[] datanodes = new DatanodeDescriptor[] {
+      new DatanodeDescriptor(
+        new DatanodeID("127.0.0.1", "localhost", "abcd", 5000, 5001, 5002)),
+      new DatanodeDescriptor(
+        new DatanodeID("127.0.1.1", "localhost", "efgh", 6000, 6001, 6002)),
+    };
+    CachedBlocksList[] lists = new CachedBlocksList[] {
+        datanodes[0].getPendingCached(),
+        datanodes[0].getCached(),
+        datanodes[1].getPendingCached(),
+        datanodes[1].getCached(),
+        datanodes[1].getPendingUncached(),
+    };
+    final int NUM_BLOCKS = 8000;
+    CachedBlock[] blocks = new CachedBlock[NUM_BLOCKS];
+    for (int i = 0; i < NUM_BLOCKS; i++) {
+      blocks[i] = new CachedBlock(i, (short)i, true);
+    }
+    Random r = new Random(654);
+    for (CachedBlocksList list : lists) {
+      testAddElementsToList(list, blocks);
+    }
+    for (CachedBlocksList list : lists) {
+      testRemoveElementsFromList(r, list, blocks);
+    }
+  }
+}

+ 319 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java

@@ -17,6 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
 import static junit.framework.Assert.assertTrue;
 import static junit.framework.Assert.fail;
 import static org.junit.Assert.assertEquals;
@@ -24,6 +30,10 @@ import static org.junit.Assert.assertFalse;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
 
 import junit.framework.Assert;
 
@@ -31,6 +41,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -46,13 +57,19 @@ import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
 import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
 import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.InvalidIdException;
 import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.NoSuchIdException;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.GSet;
 import org.junit.After;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.base.Supplier;
+
 public class TestPathBasedCacheRequests {
   static final Log LOG = LogFactory.getLog(TestPathBasedCacheRequests.class);
 
@@ -83,7 +100,7 @@ public class TestPathBasedCacheRequests {
     }
   }
 
-  @Test
+  @Test(timeout=60000)
   public void testBasicPoolOperations() throws Exception {
     final String poolName = "pool1";
     CachePoolInfo info = new CachePoolInfo(poolName).
@@ -218,7 +235,7 @@ public class TestPathBasedCacheRequests {
     dfs.addCachePool(info);
   }
 
-  @Test
+  @Test(timeout=60000)
   public void testCreateAndModifyPools() throws Exception {
     String poolName = "pool1";
     String ownerName = "abc";
@@ -301,7 +318,7 @@ public class TestPathBasedCacheRequests {
         });
   }
 
-  @Test
+  @Test(timeout=60000)
   public void testAddRemoveDirectives() throws Exception {
     proto.addCachePool(new CachePoolInfo("pool1").
         setMode(new FsPermission((short)0777)));
@@ -366,6 +383,7 @@ public class TestPathBasedCacheRequests {
     try {
       addAsUnprivileged(new PathBasedCacheDirective.Builder().
           setPath(new Path("/emptypoolname")).
+          setReplication((short)1).
           setPool("").
           build());
       Assert.fail("expected an error when adding a PathBasedCache " +
@@ -424,4 +442,302 @@ public class TestPathBasedCacheRequests {
     iter = dfs.listPathBasedCacheDescriptors(null, null);
     assertFalse(iter.hasNext());
   }
+
+  @Test(timeout=60000)
+  public void testCacheManagerRestart() throws Exception {
+    HdfsConfiguration conf = createCachingConf();
+    MiniDFSCluster cluster =
+      new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+
+    try {
+      cluster.waitActive();
+      DistributedFileSystem dfs = cluster.getFileSystem();
+
+      // Create and validate a pool
+      final String pool = "poolparty";
+      String groupName = "partygroup";
+      FsPermission mode = new FsPermission((short)0777);
+      int weight = 747;
+      dfs.addCachePool(new CachePoolInfo(pool)
+          .setGroupName(groupName)
+          .setMode(mode)
+          .setWeight(weight));
+      RemoteIterator<CachePoolInfo> pit = dfs.listCachePools();
+      assertTrue("No cache pools found", pit.hasNext());
+      CachePoolInfo info = pit.next();
+      assertEquals(pool, info.getPoolName());
+      assertEquals(groupName, info.getGroupName());
+      assertEquals(mode, info.getMode());
+      assertEquals(weight, (int)info.getWeight());
+      assertFalse("Unexpected # of cache pools found", pit.hasNext());
+  
+      // Create some cache entries
+      int numEntries = 10;
+      String entryPrefix = "/party-";
+      for (int i=0; i<numEntries; i++) {
+        dfs.addPathBasedCacheDirective(
+            new PathBasedCacheDirective.Builder().
+              setPath(new Path(entryPrefix + i)).setPool(pool).build());
+      }
+      RemoteIterator<PathBasedCacheDescriptor> dit
+          = dfs.listPathBasedCacheDescriptors(null, null);
+      for (int i=0; i<numEntries; i++) {
+        assertTrue("Unexpected # of cache entries: " + i, dit.hasNext());
+        PathBasedCacheDescriptor cd = dit.next();
+        assertEquals(i+1, cd.getEntryId());
+        assertEquals(entryPrefix + i, cd.getPath().toUri().getPath());
+        assertEquals(pool, cd.getPool());
+      }
+      assertFalse("Unexpected # of cache descriptors found", dit.hasNext());
+  
+      // Restart namenode
+      cluster.restartNameNode();
+  
+      // Check that state came back up
+      pit = dfs.listCachePools();
+      assertTrue("No cache pools found", pit.hasNext());
+      info = pit.next();
+      assertEquals(pool, info.getPoolName());
+      assertEquals(pool, info.getPoolName());
+      assertEquals(groupName, info.getGroupName());
+      assertEquals(mode, info.getMode());
+      assertEquals(weight, (int)info.getWeight());
+      assertFalse("Unexpected # of cache pools found", pit.hasNext());
+  
+      dit = dfs.listPathBasedCacheDescriptors(null, null);
+      for (int i=0; i<numEntries; i++) {
+        assertTrue("Unexpected # of cache entries: " + i, dit.hasNext());
+        PathBasedCacheDescriptor cd = dit.next();
+        assertEquals(i+1, cd.getEntryId());
+        assertEquals(entryPrefix + i, cd.getPath().toUri().getPath());
+        assertEquals(pool, cd.getPool());
+      }
+      assertFalse("Unexpected # of cache descriptors found", dit.hasNext());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private static void waitForCachedBlocks(NameNode nn,
+      final int expectedCachedBlocks, final int expectedCachedReplicas) 
+          throws Exception {
+    final FSNamesystem namesystem = nn.getNamesystem();
+    final CacheManager cacheManager = namesystem.getCacheManager();
+    LOG.info("Waiting for " + expectedCachedBlocks + " blocks with " +
+             expectedCachedReplicas + " replicas.");
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        int numCachedBlocks = 0, numCachedReplicas = 0;
+        namesystem.readLock();
+        try {
+          GSet<CachedBlock, CachedBlock> cachedBlocks =
+              cacheManager.getCachedBlocks();
+          if (cachedBlocks != null) {
+            for (Iterator<CachedBlock> iter = cachedBlocks.iterator();
+                iter.hasNext(); ) {
+              CachedBlock cachedBlock = iter.next();
+              numCachedBlocks++;
+              numCachedReplicas += cachedBlock.getDatanodes(Type.CACHED).size();
+            }
+          }
+        } finally {
+          namesystem.readUnlock();
+        }
+        if ((numCachedBlocks == expectedCachedBlocks) && 
+            (numCachedReplicas == expectedCachedReplicas)) {
+          return true;
+        } else {
+          LOG.info("cached blocks: have " + numCachedBlocks +
+              " / " + expectedCachedBlocks);
+          LOG.info("cached replicas: have " + numCachedReplicas +
+              " / " + expectedCachedReplicas);
+          return false;
+        }
+      }
+    }, 500, 60000);
+  }
+
+  private static final long BLOCK_SIZE = 512;
+  private static final int NUM_DATANODES = 4;
+
+  // Most Linux installs will allow non-root users to lock 64KB.
+  private static final long CACHE_CAPACITY = 64 * 1024 / NUM_DATANODES;
+
+  /**
+   * Return true if we can test DN caching.
+   */
+  private static boolean canTestDatanodeCaching() {
+    if (!NativeIO.isAvailable()) {
+      // Need NativeIO in order to cache blocks on the DN.
+      return false;
+    }
+    if (NativeIO.getMemlockLimit() < CACHE_CAPACITY) {
+      return false;
+    }
+    return true;
+  }
+
+  private static HdfsConfiguration createCachingConf() {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, CACHE_CAPACITY);
+    conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, true);
+    conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000);
+    conf.setLong(DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1000);
+    return conf;
+  }
+
+  @Test(timeout=120000)
+  public void testWaitForCachedReplicas() throws Exception {
+    Assume.assumeTrue(canTestDatanodeCaching());
+    HdfsConfiguration conf = createCachingConf();
+    FileSystemTestHelper helper = new FileSystemTestHelper();
+    MiniDFSCluster cluster =
+      new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
+
+    try {
+      cluster.waitActive();
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      NameNode namenode = cluster.getNameNode();
+      NamenodeProtocols nnRpc = namenode.getRpcServer();
+      Path rootDir = helper.getDefaultWorkingDirectory(dfs);
+      // Create the pool
+      final String pool = "friendlyPool";
+      nnRpc.addCachePool(new CachePoolInfo("friendlyPool"));
+      // Create some test files
+      final int numFiles = 2;
+      final int numBlocksPerFile = 2;
+      final List<String> paths = new ArrayList<String>(numFiles);
+      for (int i=0; i<numFiles; i++) {
+        Path p = new Path(rootDir, "testCachePaths-" + i);
+        FileSystemTestHelper.createFile(dfs, p, numBlocksPerFile,
+            (int)BLOCK_SIZE);
+        paths.add(p.toUri().getPath());
+      }
+      // Check the initial statistics at the namenode
+      waitForCachedBlocks(namenode, 0, 0);
+      // Cache and check each path in sequence
+      int expected = 0;
+      for (int i=0; i<numFiles; i++) {
+        PathBasedCacheDirective directive =
+            new PathBasedCacheDirective.Builder().
+              setPath(new Path(paths.get(i))).
+              setPool(pool).
+              build();
+        PathBasedCacheDescriptor descriptor =
+            nnRpc.addPathBasedCacheDirective(directive);
+        assertEquals("Descriptor does not match requested path",
+            new Path(paths.get(i)), descriptor.getPath());
+        assertEquals("Descriptor does not match requested pool", pool,
+            descriptor.getPool());
+        expected += numBlocksPerFile;
+        waitForCachedBlocks(namenode, expected, expected);
+      }
+      // Uncache and check each path in sequence
+      RemoteIterator<PathBasedCacheDescriptor> entries =
+          nnRpc.listPathBasedCacheDescriptors(0, null, null);
+      for (int i=0; i<numFiles; i++) {
+        PathBasedCacheDescriptor descriptor = entries.next();
+        nnRpc.removePathBasedCacheDescriptor(descriptor.getEntryId());
+        expected -= numBlocksPerFile;
+        waitForCachedBlocks(namenode, expected, expected);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout=120000)
+  public void testAddingPathBasedCacheDirectivesWhenCachingIsDisabled()
+      throws Exception {
+    HdfsConfiguration conf = createCachingConf();
+    conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, false);
+    MiniDFSCluster cluster =
+      new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
+
+    try {
+      cluster.waitActive();
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      NameNode namenode = cluster.getNameNode();
+      // Create the pool
+      String pool = "pool1";
+      namenode.getRpcServer().addCachePool(new CachePoolInfo(pool));
+      // Create some test files
+      final int numFiles = 2;
+      final int numBlocksPerFile = 2;
+      final List<String> paths = new ArrayList<String>(numFiles);
+      for (int i=0; i<numFiles; i++) {
+        Path p = new Path("/testCachePaths-" + i);
+        FileSystemTestHelper.createFile(dfs, p, numBlocksPerFile,
+            (int)BLOCK_SIZE);
+        paths.add(p.toUri().getPath());
+      }
+      // Check the initial statistics at the namenode
+      waitForCachedBlocks(namenode, 0, 0);
+      // Cache and check each path in sequence
+      int expected = 0;
+      for (int i=0; i<numFiles; i++) {
+        PathBasedCacheDirective directive =
+            new PathBasedCacheDirective.Builder().
+              setPath(new Path(paths.get(i))).
+              setPool(pool).
+              build();
+        dfs.addPathBasedCacheDirective(directive);
+        waitForCachedBlocks(namenode, expected, 0);
+      }
+      Thread.sleep(20000);
+      waitForCachedBlocks(namenode, expected, 0);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout=120000)
+  public void testWaitForCachedReplicasInDirectory() throws Exception {
+    Assume.assumeTrue(canTestDatanodeCaching());
+    HdfsConfiguration conf = createCachingConf();
+    MiniDFSCluster cluster =
+      new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
+
+    try {
+      cluster.waitActive();
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      NameNode namenode = cluster.getNameNode();
+      // Create the pool
+      final String pool = "friendlyPool";
+      dfs.addCachePool(new CachePoolInfo(pool));
+      // Create some test files
+      final List<Path> paths = new LinkedList<Path>();
+      paths.add(new Path("/foo/bar"));
+      paths.add(new Path("/foo/baz"));
+      paths.add(new Path("/foo2/bar2"));
+      paths.add(new Path("/foo2/baz2"));
+      dfs.mkdir(new Path("/foo"), FsPermission.getDirDefault());
+      dfs.mkdir(new Path("/foo2"), FsPermission.getDirDefault());
+      final int numBlocksPerFile = 2;
+      for (Path path : paths) {
+        FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
+            (int)BLOCK_SIZE, (short)3, false);
+      }
+      waitForCachedBlocks(namenode, 0, 0);
+      // cache entire directory
+      PathBasedCacheDescriptor descriptor = dfs.addPathBasedCacheDirective(
+            new PathBasedCacheDirective.Builder().
+              setPath(new Path("/foo")).
+              setReplication((short)2).
+              setPool(pool).
+              build());
+      assertEquals("Descriptor does not match requested pool", pool,
+          descriptor.getPool());
+      waitForCachedBlocks(namenode, 4, 8);
+      // remove and watch numCached go to 0
+      dfs.removePathBasedCacheDescriptor(descriptor);
+      waitForCachedBlocks(namenode, 0, 0);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
 }