Browse Source

Merge forward from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1552205 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 years ago
parent
commit
19fb4cb2d1
100 changed files with 4316 additions and 2622 deletions
  1. 18 2
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
  3. 16 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
  4. 14 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcConstants.java
  5. 127 44
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  6. 51 36
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  7. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
  8. 1 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  9. 19 4
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
  10. 61 14
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  11. 1 8
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  12. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/CacheFlag.java
  13. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
  14. 18 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
  15. 530 393
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  16. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
  17. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  18. 71 46
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  19. 12 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  20. 25 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  21. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
  22. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  23. 12 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  24. 10 82
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
  25. 17 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
  26. 9 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  27. 7 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  28. 19 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  29. 26 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  30. 10 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  31. 186 41
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  32. 15 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  33. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  34. 22 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  35. 26 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  36. 24 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/CachingStrategy.java
  37. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  38. 2 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  39. 199 86
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  40. 25 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
  41. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DirectoryWithQuotaFeature.java
  42. 29 40
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  43. 4 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  44. 11 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  45. 33 186
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  46. 13 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
  47. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  48. 206 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  49. 14 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  50. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
  51. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
  52. 22 65
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  53. 265 126
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  54. 16 39
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  55. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  56. 33 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  57. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  58. 47 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
  59. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
  60. 7 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  61. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  62. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  63. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  64. 321 462
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  65. 2 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  66. 27 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  67. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
  68. 11 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
  69. 66 49
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
  70. 10 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  71. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  72. 10 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  73. 3 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  74. 310 46
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
  75. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java
  76. 81 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  77. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
  78. 135 36
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
  79. 87 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java
  80. 213 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
  81. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
  82. 341 305
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  83. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  84. 30 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  85. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
  86. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  87. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
  88. 9 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
  89. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
  90. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java
  91. 45 58
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  92. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
  93. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  94. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java
  95. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  96. 92 94
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  97. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml
  98. 18 0
      hadoop-mapreduce-project/CHANGES.txt
  99. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  100. 20 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

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

@@ -280,6 +280,8 @@ Trunk (Unreleased)
     HDFS-5471. CacheAdmin -listPools fails when user lacks permissions to view
     all pools (Andrew Wang via Colin Patrick McCabe)
 
+    HADOOP-10044 Improve the javadoc of rpc code (sanjay Radia)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -395,12 +397,16 @@ Release 2.4.0 - UNRELEASED
 
     HADOOP-10102. Update commons IO from 2.1 to 2.4 (Akira Ajisaka via stevel)
 
+    HADOOP-10168. fix javadoc of ReflectionUtils#copy. (Thejas Nair via suresh)
+
+    HADOOP-10164. Allow UGI to login with a known Subject (bobby)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
 
-   HADOOP-10047. Add a direct-buffer based apis for compression. (Gopal V
-   via acmurthy)
+    HADOOP-10047. Add a direct-buffer based apis for compression. (Gopal V
+    via acmurthy)
 
   BUG FIXES
 
@@ -465,6 +471,16 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10058. TestMetricsSystemImpl#testInitFirstVerifyStopInvokedImmediately
     fails on trunk (Chen He via jeagles)
 
+    HADOOP-8753. LocalDirAllocator throws "ArithmeticException: / by zero" when
+    there is no available space on configured local dir. (Benoy Antony via hitesh)
+
+    HADOOP-10106. Incorrect thread name in RPC log messages. (Ming Ma via jing9)
+
+    HADOOP-9611 mvn-rpmbuild against google-guice > 3.0 yields missing cglib
+    dependency (Timothy St. Clair via stevel)
+
+    HADOOP-10171. TestRPC fails intermittently on jkd7 (Mit Desai via jeagles)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

+ 61 - 14
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -216,39 +216,37 @@ Trunk (Unreleased)
     and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
     (jing9 via szetszwo)
 
-    HDFS-5538. URLConnectionFactory should pick up the SSL related configuration 
-    by default. (Haohui Mai via jing9)
-
     HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
     with DirectoryWithQuotaFeature.  (szetszwo)
 
     HDFS-5556. Add some more NameNode cache statistics, cache pool stats
     (cmccabe)
 
-    HDFS-5545. Allow specifying endpoints for listeners in HttpServer. (Haohui
-    Mai via jing9)
-
     HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
 
     HDFS-5430. Support TTL on CacheDirectives. (wang)
 
-    HDFS-5536. Implement HTTP policy for Namenode and DataNode. (Haohui Mai via
-    jing9)
-
     HDFS-5630. Hook up cache directive and pool usage statistics. (wang)
 
-    HDFS-5312. Generate HTTP / HTTPS URL in DFSUtil#getInfoServer() based on the 
-    configured http policy. (Haohui Mai via jing9)
-
     HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
     FileWithSnapshotFeature.  (jing9 via szetszwo)
 
-    HDFS-5629. Support HTTPS in JournalNode and SecondaryNameNode. 
-    (Haohui Mai via jing9)
+    HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
+    via jing9)
+
+    HDFS-5632. Flatten INodeDirectory hierarchy: Replace
+    INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
+    (jing9 via szetszwo)
+
+    HDFS-5431. Support cachepool-based limit management in path-based caching
+    (awang via cmccabe)
 
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
+    HDFS-5665. Remove the unnecessary writeLock while initializing CacheManager
+    in FsNameSystem Ctor. (Uma Maheswara Rao G via Andrew Wang)
+
   BUG FIXES
     HADOOP-9635 Fix potential Stack Overflow in DomainSocket.c (V. Karthik Kumar
                 via cmccabe)
@@ -444,6 +442,12 @@ Trunk (Unreleased)
 
     HDFS-5626. dfsadmin -report shows incorrect cache values. (cmccabe)
 
+    HDFS-5406. Send incremental block reports for all storages in a
+    single call. (Arpit Agarwal)
+
+    HDFS-5454. DataNode UUID should be assigned prior to FsDataset
+    initialization. (Arpit Agarwal)
+
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
 
     HDFS-4985. Add storage type to the protocol and expose it in block report
@@ -722,6 +726,34 @@ Release 2.4.0 - UNRELEASED
     HDFS-5637. Try to refeatchToken while local read InvalidToken occurred.
     (Liang Xie via junping_du)
 
+    HDFS-5652. Refactor invalid block token exception handling in DFSInputStream.
+    (Liang Xie via junping_du)
+
+    HDFS-5350. Name Node should report fsimage transfer time as a metric.
+    (Jimmy Xiang via wang)
+
+    HDFS-5538. URLConnectionFactory should pick up the SSL related configuration 
+    by default. (Haohui Mai via jing9)
+
+    HDFS-5545. Allow specifying endpoints for listeners in HttpServer. (Haohui
+    Mai via jing9)
+
+    HDFS-5536. Implement HTTP policy for Namenode and DataNode. (Haohui Mai via
+    jing9)
+
+    HDFS-5312. Generate HTTP / HTTPS URL in DFSUtil#getInfoServer() based on the 
+    configured http policy. (Haohui Mai via jing9)
+
+    HDFS-5629. Support HTTPS in JournalNode and SecondaryNameNode. 
+    (Haohui Mai via jing9)    
+
+    HDFS-5674. Editlog code cleanup: remove @SuppressWarnings("deprecation") in
+    FSEditLogOp; change FSEditLogOpCodes.fromByte(..) to be more efficient; and
+    change Some fields in FSEditLog to final.  (szetszwo)
+
+    HDFS-5634. Allow BlockReaderLocal to switch between checksumming and not
+    (cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -777,6 +809,8 @@ Release 2.4.0 - UNRELEASED
     HDFS-5580. Fix infinite loop in Balancer.waitForMoveCompletion.
     (Binglin Chang via junping_du)
 
+    HDFS-5676. fix inconsistent synchronization of CachingStrategy (cmccabe)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -804,6 +838,12 @@ Release 2.3.0 - UNRELEASED
     HDFS-4983. Numeric usernames do not work with WebHDFS FS. (Yongjun Zhang via
     jing9)
 
+    HDFS-5592. statechangeLog of completeFile should be logged only in case of success. 
+    (Vinayakumar via umamahesh)
+
+    HDFS-5662. Can't decommission a DataNode due to file's replication factor
+    larger than the rest of the cluster size. (brandonli)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -947,6 +987,13 @@ Release 2.3.0 - UNRELEASED
 
     HDFS-4201. NPE in BPServiceActor#sendHeartBeat. (jxiang via cmccabe)
 
+    HDFS-5666. Fix inconsistent synchronization in BPOfferService (jxiang via cmccabe)
+    
+    HDFS-5657. race condition causes writeback state error in NFS gateway (brandonli)
+
+    HDFS-5661. Browsing FileSystem via web ui, should use datanode's fqdn instead of ip 
+    address. (Benoy Antony via jing9)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

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

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

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

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Specifies semantics for CacheDirective operations. Multiple flags can
+ * be combined in an EnumSet.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum CacheFlag {
+
+  /**
+   * Ignore cache pool resource limits when performing this operation.
+   */
+  FORCE((short) 0x01);
+  private final short mode;
+
+  private CacheFlag(short mode) {
+    this.mode = mode;
+  }
+
+  short getMode() {
+    return mode;
+  }
+}

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

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

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

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

File diff suppressed because it is too large
+ 530 - 393
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java


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

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

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -85,6 +85,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStorageLocation;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -2295,20 +2296,20 @@ public class DFSClient implements java.io.Closeable {
   }
 
   public long addCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
     try {
-      return namenode.addCacheDirective(info);
+      return namenode.addCacheDirective(info, flags);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }
   }
   
   public void modifyCacheDirective(
-      CacheDirectiveInfo info) throws IOException {
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
     try {
-      namenode.modifyCacheDirective(info);
+      namenode.modifyCacheDirective(info, flags);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }

+ 71 - 46
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.io.ByteBufferPool;
@@ -227,7 +228,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         dfsClient.getConf().shortCircuitStreamsCacheSize,
         dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
     this.cachingStrategy =
-        dfsClient.getDefaultReadCachingStrategy().duplicate();
+        dfsClient.getDefaultReadCachingStrategy();
     openInfo();
   }
 
@@ -573,7 +574,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
         blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
             accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
-            buffersize, verifyChecksum, dfsClient.clientName);
+            buffersize, verifyChecksum, dfsClient.clientName, cachingStrategy);
         if(connectFailedOnce) {
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
                              " for " + blk);
@@ -591,21 +592,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // The encryption key used is invalid.
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
-        } else if ((ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken)
-            && refetchToken > 0) {
-          DFSClient.LOG.info("Will fetch a new access token and retry, " 
-              + "access token was invalid when connecting to " + targetAddr
-              + " : " + ex);
-          /*
-           * Get a new access token and retry. Retry is needed in 2 cases. 1)
-           * When both NN and DN re-started while DFSClient holding a cached
-           * access token. 2) In the case that NN fails to update its
-           * access key at pre-set interval (by a wide margin) and
-           * subsequently restarts. In this case, DN re-registers itself with
-           * NN and receives a new access key, but DN will delete the old
-           * access key from its memory since it's considered expired based on
-           * the estimated expiration date.
-           */
+        } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
           refetchToken--;
           fetchBlockAt(target);
         } else {
@@ -941,7 +928,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       // cached block locations may have been updated by chooseDataNode()
       // or fetchBlockAt(). Always get the latest list of locations at the 
       // start of the loop.
-      block = getBlockAt(block.getStartOffset(), false);
+      CachingStrategy curCachingStrategy;
+      synchronized (this) {
+        block = getBlockAt(block.getStartOffset(), false);
+        curCachingStrategy = cachingStrategy;
+      }
       DNAddrPair retval = chooseDataNode(block);
       DatanodeInfo chosenNode = retval.info;
       InetSocketAddress targetAddr = retval.addr;
@@ -953,7 +944,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         int len = (int) (end - start + 1);
         reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
             blockToken, start, len, buffersize, verifyChecksum,
-            dfsClient.clientName);
+            dfsClient.clientName, curCachingStrategy);
         int nread = reader.readAll(buf, offset, len);
         if (nread != len) {
           throw new IOException("truncated return from reader.read(): " +
@@ -978,11 +969,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // The encryption key used is invalid.
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
-        } else if ((e instanceof InvalidBlockTokenException || e instanceof InvalidToken)
-            && refetchToken > 0) {
-          DFSClient.LOG.info("Will get a new access token and retry, "
-              + "access token was invalid when connecting to " + targetAddr
-              + " : " + e);
+        } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
           refetchToken--;
           fetchBlockAt(block.getStartOffset());
           continue;
@@ -1003,6 +990,34 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
   }
 
+  /**
+   * Should the block access token be refetched on an exception
+   * 
+   * @param ex Exception received
+   * @param targetAddr Target datanode address from where exception was received
+   * @return true if block access token has expired or invalid and it should be
+   *         refetched
+   */
+  private static boolean tokenRefetchNeeded(IOException ex,
+      InetSocketAddress targetAddr) {
+    /*
+     * Get a new access token and retry. Retry is needed in 2 cases. 1)
+     * When both NN and DN re-started while DFSClient holding a cached
+     * access token. 2) In the case that NN fails to update its
+     * access key at pre-set interval (by a wide margin) and
+     * subsequently restarts. In this case, DN re-registers itself with
+     * NN and receives a new access key, but DN will delete the old
+     * access key from its memory since it's considered expired based on
+     * the estimated expiration date.
+     */
+    if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
+      DFSClient.LOG.info("Access token was invalid when connecting to "
+          + targetAddr + " : " + ex);
+      return true;
+    }
+    return false;
+  }
+
   private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
     Peer peer = null;
     boolean success = false;
@@ -1042,6 +1057,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * @param bufferSize  The IO buffer size (not the client buffer size)
    * @param verifyChecksum  Whether to verify checksum
    * @param clientName  Client name
+   * @param CachingStrategy  caching strategy to use
    * @return New BlockReader instance
    */
   protected BlockReader getBlockReader(InetSocketAddress dnAddr,
@@ -1053,7 +1069,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
                                        long len,
                                        int bufferSize,
                                        boolean verifyChecksum,
-                                       String clientName)
+                                       String clientName,
+                                       CachingStrategy curCachingStrategy)
       throws IOException {
     // Firstly, we check to see if we have cached any file descriptors for
     // local blocks.  If so, we can just re-use those file descriptors.
@@ -1063,9 +1080,18 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         DFSClient.LOG.debug("got FileInputStreams for " + block + " from " +
             "the FileInputStreamCache.");
       }
-      return new BlockReaderLocal(dfsClient.getConf(), file,
-        block, startOffset, len, fis[0], fis[1], chosenNode, verifyChecksum,
-        fileInputStreamCache);
+      return new BlockReaderLocal.Builder(dfsClient.getConf()).
+          setFilename(file).
+          setBlock(block).
+          setStartOffset(startOffset).
+          setStreams(fis).
+          setDatanodeID(chosenNode).
+          setVerifyChecksum(verifyChecksum).
+          setBlockMetadataHeader(BlockMetadataHeader.
+              preadHeader(fis[1].getChannel())).
+          setFileInputStreamCache(fileInputStreamCache).
+          setCachingStrategy(curCachingStrategy).
+          build();
     }
     
     // If the legacy local block reader is enabled and we are reading a local
@@ -1099,7 +1125,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             len, verifyChecksum, clientName, peer, chosenNode, 
             dsFactory, peerCache, fileInputStreamCache,
-            allowShortCircuitLocalReads, cachingStrategy);
+            allowShortCircuitLocalReads, curCachingStrategy);
         return reader;
       } catch (IOException ex) {
         DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
@@ -1122,7 +1148,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             len, verifyChecksum, clientName, peer, chosenNode,
             dsFactory, peerCache, fileInputStreamCache,
-            allowShortCircuitLocalReads, cachingStrategy);
+            allowShortCircuitLocalReads, curCachingStrategy);
         return reader;
       } catch (IOException e) {
         DFSClient.LOG.warn("failed to connect to " + domSock, e);
@@ -1146,7 +1172,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             dfsClient.getConf(), file, block, blockToken, startOffset,
             len, verifyChecksum, clientName, peer, chosenNode, 
             dsFactory, peerCache, fileInputStreamCache, false,
-            cachingStrategy);
+            curCachingStrategy);
         return reader;
       } catch (IOException ex) {
         DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
@@ -1166,7 +1192,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         dfsClient.getConf(), file, block, blockToken, startOffset,
         len, verifyChecksum, clientName, peer, chosenNode, 
         dsFactory, peerCache, fileInputStreamCache, false,
-        cachingStrategy);
+        curCachingStrategy);
   }
 
 
@@ -1440,14 +1466,18 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   @Override
   public synchronized void setReadahead(Long readahead)
       throws IOException {
-    this.cachingStrategy.setReadahead(readahead);
+    this.cachingStrategy =
+        new CachingStrategy.Builder(this.cachingStrategy).
+            setReadahead(readahead).build();
     closeCurrentBlockReader();
   }
 
   @Override
   public synchronized void setDropBehind(Boolean dropBehind)
       throws IOException {
-    this.cachingStrategy.setDropBehind(dropBehind);
+    this.cachingStrategy =
+        new CachingStrategy.Builder(this.cachingStrategy).
+            setDropBehind(dropBehind).build();
     closeCurrentBlockReader();
   }
 
@@ -1469,23 +1499,19 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             "at position " + pos);
       }
     }
-    boolean canSkipChecksums = opts.contains(ReadOption.SKIP_CHECKSUMS);
-    if (canSkipChecksums) {
-      ByteBuffer buffer = tryReadZeroCopy(maxLength);
-      if (buffer != null) {
-        return buffer;
-      }
+    ByteBuffer buffer = tryReadZeroCopy(maxLength, opts);
+    if (buffer != null) {
+      return buffer;
     }
-    ByteBuffer buffer = ByteBufferUtil.
-        fallbackRead(this, bufferPool, maxLength);
+    buffer = ByteBufferUtil.fallbackRead(this, bufferPool, maxLength);
     if (buffer != null) {
       extendedReadBuffers.put(buffer, bufferPool);
     }
     return buffer;
   }
 
-  private synchronized ByteBuffer tryReadZeroCopy(int maxLength)
-      throws IOException {
+  private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
+      EnumSet<ReadOption> opts) throws IOException {
     // Java ByteBuffers can't be longer than 2 GB, because they use
     // 4-byte signed integers to represent capacity, etc.
     // So we can't mmap the parts of the block higher than the 2 GB offset.
@@ -1508,8 +1534,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     long blockPos = curPos - blockStartInFile;
     long limit = blockPos + length;
     ClientMmap clientMmap =
-        blockReader.getClientMmap(currentLocatedBlock,
-            dfsClient.getMmapManager());
+        blockReader.getClientMmap(opts, dfsClient.getMmapManager());
     if (clientMmap == null) {
       if (DFSClient.LOG.isDebugEnabled()) {
         DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +

+ 12 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -150,7 +150,7 @@ public class DFSOutputStream extends FSOutputSummer
   private Progressable progress;
   private final short blockReplication; // replication factor of file
   private boolean shouldSyncBlock = false; // force blocks to disk upon close
-  private CachingStrategy cachingStrategy;
+  private AtomicReference<CachingStrategy> cachingStrategy;
   private boolean failPacket = false;
   
   private static class Packet {
@@ -1183,7 +1183,7 @@ public class DFSOutputStream extends FSOutputSummer
           new Sender(out).writeBlock(block, accessToken, dfsClient.clientName,
               nodes, null, recoveryFlag? stage.getRecoveryStage() : stage, 
               nodes.length, block.getNumBytes(), bytesSent, newGS, checksum,
-              cachingStrategy);
+              cachingStrategy.get());
   
           // receive ack for connect
           BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
@@ -1378,8 +1378,8 @@ public class DFSOutputStream extends FSOutputSummer
     this.blockSize = stat.getBlockSize();
     this.blockReplication = stat.getReplication();
     this.progress = progress;
-    this.cachingStrategy =
-        dfsClient.getDefaultWriteCachingStrategy().duplicate();
+    this.cachingStrategy = new AtomicReference<CachingStrategy>(
+        dfsClient.getDefaultWriteCachingStrategy());
     if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug(
           "Set non-null progress callback on DFSOutputStream " + src);
@@ -1993,7 +1993,14 @@ public class DFSOutputStream extends FSOutputSummer
 
   @Override
   public void setDropBehind(Boolean dropBehind) throws IOException {
-    this.cachingStrategy.setDropBehind(dropBehind);
+    CachingStrategy prevStrategy, nextStrategy;
+    // CachingStrategy is immutable.  So build a new CachingStrategy with the
+    // modifications we want, and compare-and-swap it in.
+    do {
+      prevStrategy = this.cachingStrategy.get();
+      nextStrategy = new CachingStrategy.Builder(prevStrategy).
+                        setDropBehind(dropBehind).build();
+    } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy));
   }
 
   @VisibleForTesting

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

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

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

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

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

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

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

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

+ 10 - 82
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hdfs.protocol;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 
 import javax.annotation.Nullable;
@@ -32,14 +30,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
-import org.apache.hadoop.hdfs.util.XMLUtils;
-import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
-import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
-import org.apache.hadoop.io.Text;
-import org.xml.sax.ContentHandler;
-import org.xml.sax.SAXException;
 
 /**
  * CachePoolInfo describes a cache pool.
@@ -64,7 +54,7 @@ public class CachePoolInfo {
   FsPermission mode;
 
   @Nullable
-  Integer weight;
+  Long limit;
 
   public CachePoolInfo(String poolName) {
     this.poolName = poolName;
@@ -101,12 +91,12 @@ public class CachePoolInfo {
     return this;
   }
 
-  public Integer getWeight() {
-    return weight;
+  public Long getLimit() {
+    return limit;
   }
 
-  public CachePoolInfo setWeight(Integer weight) {
-    this.weight = weight;
+  public CachePoolInfo setLimit(Long bytes) {
+    this.limit = bytes;
     return this;
   }
 
@@ -117,7 +107,7 @@ public class CachePoolInfo {
       append(", groupName:").append(groupName).
       append(", mode:").append((mode == null) ? "null" :
           String.format("0%03o", mode.toShort())).
-      append(", weight:").append(weight).
+      append(", limit:").append(limit).
       append("}").toString();
   }
   
@@ -134,7 +124,7 @@ public class CachePoolInfo {
         append(ownerName, other.ownerName).
         append(groupName, other.groupName).
         append(mode, other.mode).
-        append(weight, other.weight).
+        append(limit, other.limit).
         isEquals();
   }
 
@@ -145,7 +135,7 @@ public class CachePoolInfo {
         append(ownerName).
         append(groupName).
         append(mode).
-        append(weight).
+        append(limit).
         hashCode();
   }
 
@@ -153,8 +143,8 @@ public class CachePoolInfo {
     if (info == null) {
       throw new InvalidRequestException("CachePoolInfo is null");
     }
-    if ((info.getWeight() != null) && (info.getWeight() < 0)) {
-      throw new InvalidRequestException("CachePool weight is negative.");
+    if ((info.getLimit() != null) && (info.getLimit() < 0)) {
+      throw new InvalidRequestException("Limit is negative.");
     }
     validateName(info.poolName);
   }
@@ -167,66 +157,4 @@ public class CachePoolInfo {
       throw new IOException("invalid empty cache pool name");
     }
   }
-
-  public static CachePoolInfo readFrom(DataInput in) throws IOException {
-    String poolName = Text.readString(in);
-    CachePoolInfo info = new CachePoolInfo(poolName);
-    if (in.readBoolean()) {
-      info.setOwnerName(Text.readString(in));
-    }
-    if (in.readBoolean())  {
-      info.setGroupName(Text.readString(in));
-    }
-    if (in.readBoolean()) {
-      info.setMode(FsPermission.read(in));
-    }
-    if (in.readBoolean()) {
-      info.setWeight(in.readInt());
-    }
-    return info;
-  }
-
-  public void writeTo(DataOutput out) throws IOException {
-    Text.writeString(out, poolName);
-    boolean hasOwner, hasGroup, hasMode, hasWeight;
-    hasOwner = ownerName != null;
-    hasGroup = groupName != null;
-    hasMode = mode != null;
-    hasWeight = weight != null;
-    out.writeBoolean(hasOwner);
-    if (hasOwner) {
-      Text.writeString(out, ownerName);
-    }
-    out.writeBoolean(hasGroup);
-    if (hasGroup) {
-      Text.writeString(out, groupName);
-    }
-    out.writeBoolean(hasMode);
-    if (hasMode) {
-      mode.write(out);
-    }
-    out.writeBoolean(hasWeight);
-    if (hasWeight) {
-      out.writeInt(weight);
-    }
-  }
-
-  public void writeXmlTo(ContentHandler contentHandler) throws SAXException {
-    XMLUtils.addSaxString(contentHandler, "POOLNAME", poolName);
-    PermissionStatus perm = new PermissionStatus(ownerName,
-        groupName, mode);
-    FSEditLogOp.permissionStatusToXml(contentHandler, perm);
-    XMLUtils.addSaxString(contentHandler, "WEIGHT", Integer.toString(weight));
-  }
-
-  public static CachePoolInfo readXmlFrom(Stanza st) throws InvalidXmlException {
-    String poolName = st.getValue("POOLNAME");
-    PermissionStatus perm = FSEditLogOp.permissionStatusFromXml(st);
-    int weight = Integer.parseInt(st.getValue("WEIGHT"));
-    return new CachePoolInfo(poolName).
-        setOwnerName(perm.getUserName()).
-        setGroupName(perm.getGroupName()).
-        setMode(perm.getPermission()).
-        setWeight(weight);
-  }
 }

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

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

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

@@ -19,9 +19,11 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -1100,23 +1102,24 @@ public interface ClientProtocol {
    * Add a CacheDirective to the CacheManager.
    * 
    * @param directive A CacheDirectiveInfo to be added
+   * @param flags {@link CacheFlag}s to use for this operation.
    * @return A CacheDirectiveInfo associated with the added directive
    * @throws IOException if the directive could not be added
    */
   @AtMostOnce
-  public long addCacheDirective(
-      CacheDirectiveInfo directive) throws IOException;
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
 
   /**
    * Modify a CacheDirective in the CacheManager.
    * 
-   * @return directive The directive to modify.  Must contain 
-   *                   a directive ID.
+   * @return directive The directive to modify. Must contain a directive ID.
+   * @param flags {@link CacheFlag}s to use for this operation.
    * @throws IOException if the directive could not be modified
    */
   @AtMostOnce
-  public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException;
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
 
   /**
    * Remove a CacheDirectiveInfo from the CacheManager.

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

@@ -320,7 +320,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     try {
       HdfsFileStatus result = server.create(req.getSrc(),
           PBHelper.convert(req.getMasked()), req.getClientName(),
-          PBHelper.convert(req.getCreateFlag()), req.getCreateParent(),
+          PBHelper.convertCreateFlag(req.getCreateFlag()), req.getCreateParent(),
           (short) req.getReplication(), req.getBlockSize());
 
       if (result != null) {
@@ -1034,9 +1034,11 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, AddCacheDirectiveRequestProto request)
       throws ServiceException {
     try {
+      long id = server.addCacheDirective(
+          PBHelper.convert(request.getInfo()),
+          PBHelper.convertCacheFlags(request.getCacheFlags()));
       return AddCacheDirectiveResponseProto.newBuilder().
-              setId(server.addCacheDirective(
-                  PBHelper.convert(request.getInfo()))).build();
+              setId(id).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1048,7 +1050,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     try {
       server.modifyCacheDirective(
-          PBHelper.convert(request.getInfo()));
+          PBHelper.convert(request.getInfo()),
+          PBHelper.convertCacheFlags(request.getCacheFlags()));
       return ModifyCacheDirectiveResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);

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

@@ -21,10 +21,12 @@ import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -1003,24 +1005,32 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public long addCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
     try {
-      return rpcProxy.addCacheDirective(null, 
-              AddCacheDirectiveRequestProto.newBuilder().
-                  setInfo(PBHelper.convert(directive)).build()).getId();
+      AddCacheDirectiveRequestProto.Builder builder =
+          AddCacheDirectiveRequestProto.newBuilder().
+              setInfo(PBHelper.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelper.convertCacheFlags(flags));
+      }
+      return rpcProxy.addCacheDirective(null, builder.build()).getId();
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
 
   @Override
-  public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
     try {
-      rpcProxy.modifyCacheDirective(null,
+      ModifyCacheDirectiveRequestProto.Builder builder =
           ModifyCacheDirectiveRequestProto.newBuilder().
-              setInfo(PBHelper.convert(directive)).build());
+              setInfo(PBHelper.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelper.convertCacheFlags(flags));
+      }
+      rpcProxy.modifyCacheDirective(null, builder.build());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

+ 26 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -27,6 +27,7 @@ import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -64,6 +65,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
@@ -1182,7 +1184,7 @@ public class PBHelper {
     return value;
   }
   
-  public static EnumSetWritable<CreateFlag> convert(int flag) {
+  public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
     EnumSet<CreateFlag> result = 
        EnumSet.noneOf(CreateFlag.class);   
     if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
@@ -1197,7 +1199,23 @@ public class PBHelper {
     }
     return new EnumSetWritable<CreateFlag>(result);
   }
-  
+
+  public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
+    int value = 0;
+    if (flags.contains(CacheFlag.FORCE)) {
+      value |= CacheFlagProto.FORCE.getNumber();
+    }
+    return value;
+  }
+
+  public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
+    EnumSet<CacheFlag> result = EnumSet.noneOf(CacheFlag.class);
+    if ((flags & CacheFlagProto.FORCE_VALUE) == CacheFlagProto.FORCE_VALUE) {
+      result.add(CacheFlag.FORCE);
+    }
+    return result;
+  }
+
   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
     if (fs == null)
       return null;
@@ -1795,8 +1813,8 @@ public class PBHelper {
     if (info.getMode() != null) {
       builder.setMode(info.getMode().toShort());
     }
-    if (info.getWeight() != null) {
-      builder.setWeight(info.getWeight());
+    if (info.getLimit() != null) {
+      builder.setLimit(info.getLimit());
     }
     return builder.build();
   }
@@ -1814,8 +1832,8 @@ public class PBHelper {
     if (proto.hasMode()) {
       info.setMode(new FsPermission((short)proto.getMode()));
     }
-    if (proto.hasWeight()) {
-      info.setWeight(proto.getWeight());
+    if (proto.hasLimit())  {
+      info.setLimit(proto.getLimit());
     }
     return info;
   }
@@ -1824,6 +1842,7 @@ public class PBHelper {
     CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
     builder.setBytesNeeded(stats.getBytesNeeded());
     builder.setBytesCached(stats.getBytesCached());
+    builder.setBytesOverlimit(stats.getBytesOverlimit());
     builder.setFilesNeeded(stats.getFilesNeeded());
     builder.setFilesCached(stats.getFilesCached());
     return builder.build();
@@ -1833,6 +1852,7 @@ public class PBHelper {
     CachePoolStats.Builder builder = new CachePoolStats.Builder();
     builder.setBytesNeeded(proto.getBytesNeeded());
     builder.setBytesCached(proto.getBytesCached());
+    builder.setBytesOverlimit(proto.getBytesOverlimit());
     builder.setFilesNeeded(proto.getFilesNeeded());
     builder.setFilesCached(proto.getFilesCached());
     return builder.build();

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

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

+ 186 - 41
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java

@@ -27,6 +27,9 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -48,6 +51,8 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.Time;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Scans the namesystem, scheduling blocks to be cached as appropriate.
  *
@@ -79,25 +84,52 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
   private final long intervalMs;
 
   /**
-   * True if we should rescan immediately, regardless of how much time
-   * elapsed since the previous scan.
+   * The CacheReplicationMonitor (CRM) lock. Used to synchronize starting and
+   * waiting for rescan operations.
    */
-  private boolean rescanImmediately;
+  private final ReentrantLock lock = new ReentrantLock();
 
   /**
-   * The monotonic time at which the current scan started.
+   * Notifies the scan thread that an immediate rescan is needed.
    */
-  private long scanTimeMs;
+  private final Condition doRescan = lock.newCondition();
 
   /**
-   * Mark status of the current scan.
+   * Notifies waiting threads that a rescan has finished.
    */
-  private boolean mark = false;
+  private final Condition scanFinished = lock.newCondition();
+
+  /**
+   * Whether there are pending CacheManager operations that necessitate a
+   * CacheReplicationMonitor rescan. Protected by the CRM lock.
+   */
+  private boolean needsRescan = true;
+
+  /**
+   * Whether we are currently doing a rescan. Protected by the CRM lock.
+   */
+  private boolean isScanning = false;
+
+  /**
+   * The number of rescans completed. Used to wait for scans to finish.
+   * Protected by the CacheReplicationMonitor lock.
+   */
+  private long scanCount = 0;
+
+  /**
+   * True if this monitor should terminate. Protected by the CRM lock.
+   */
+  private boolean shutdown = false;
 
   /**
-   * True if this monitor should terminate.
+   * The monotonic time at which the current scan started.
+   */
+  private long startTimeMs;
+
+  /**
+   * Mark status of the current scan.
    */
-  private boolean shutdown;
+  private boolean mark = false;
 
   /**
    * Cache directives found in the previous scan.
@@ -108,7 +140,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
    * Blocks found in the previous scan.
    */
   private long scannedBlocks;
-  
+
   public CacheReplicationMonitor(FSNamesystem namesystem,
       CacheManager cacheManager, long intervalMs) {
     this.namesystem = namesystem;
@@ -120,41 +152,60 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
 
   @Override
   public void run() {
-    shutdown = false;
-    rescanImmediately = true;
-    scanTimeMs = 0;
+    startTimeMs = 0;
     LOG.info("Starting CacheReplicationMonitor with interval " +
              intervalMs + " milliseconds");
     try {
       long curTimeMs = Time.monotonicNow();
       while (true) {
-        synchronized(this) {
+        // Not all of the variables accessed here need the CRM lock, but take
+        // it anyway for simplicity
+        lock.lock();
+        try {
           while (true) {
             if (shutdown) {
               LOG.info("Shutting down CacheReplicationMonitor");
               return;
             }
-            if (rescanImmediately) {
-              LOG.info("Rescanning on request");
-              rescanImmediately = false;
+            if (needsRescan) {
+              LOG.info("Rescanning because of pending operations");
               break;
             }
-            long delta = (scanTimeMs + intervalMs) - curTimeMs;
+            long delta = (startTimeMs + intervalMs) - curTimeMs;
             if (delta <= 0) {
-              LOG.info("Rescanning after " + (curTimeMs - scanTimeMs) +
+              LOG.info("Rescanning after " + (curTimeMs - startTimeMs) +
                   " milliseconds");
               break;
             }
-            this.wait(delta);
+            doRescan.await(delta, TimeUnit.MILLISECONDS);
             curTimeMs = Time.monotonicNow();
           }
+        } finally {
+          lock.unlock();
         }
-        scanTimeMs = curTimeMs;
+        // Mark scan as started, clear needsRescan
+        lock.lock();
+        try {
+          isScanning = true;
+          needsRescan = false;
+        } finally {
+          lock.unlock();
+        }
+        startTimeMs = curTimeMs;
         mark = !mark;
         rescan();
         curTimeMs = Time.monotonicNow();
+        // Retake the CRM lock to update synchronization-related variables
+        lock.lock();
+        try {
+          isScanning = false;
+          scanCount++;
+          scanFinished.signalAll();
+        } finally {
+          lock.unlock();
+        }
         LOG.info("Scanned " + scannedDirectives + " directive(s) and " +
-            scannedBlocks + " block(s) in " + (curTimeMs - scanTimeMs) + " " +
+            scannedBlocks + " block(s) in " + (curTimeMs - startTimeMs) + " " +
             "millisecond(s).");
       }
     } catch (Throwable t) {
@@ -164,15 +215,91 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
   }
 
   /**
-   * Kick the monitor thread.
-   * 
-   * If it is sleeping, it will wake up and start scanning.
-   * If it is currently scanning, it will finish the scan and immediately do 
-   * another one.
+   * Similar to {@link CacheReplicationMonitor#waitForRescan()}, except it only
+   * waits if there are pending operations that necessitate a rescan as
+   * indicated by {@link #setNeedsRescan()}.
+   * <p>
+   * Note that this call may release the FSN lock, so operations before and
+   * after are not necessarily atomic.
+   */
+  public void waitForRescanIfNeeded() {
+    lock.lock();
+    try {
+      if (!needsRescan) {
+        return;
+      }
+    } finally {
+      lock.unlock();
+    }
+    waitForRescan();
+  }
+
+  /**
+   * Waits for a rescan to complete. This doesn't guarantee consistency with
+   * pending operations, only relative recency, since it will not force a new
+   * rescan if a rescan is already underway.
+   * <p>
+   * Note that this call will release the FSN lock, so operations before and
+   * after are not atomic.
+   */
+  public void waitForRescan() {
+    // Drop the FSN lock temporarily and retake it after we finish waiting
+    // Need to handle both the read lock and the write lock
+    boolean retakeWriteLock = false;
+    if (namesystem.hasWriteLock()) {
+      namesystem.writeUnlock();
+      retakeWriteLock = true;
+    } else if (namesystem.hasReadLock()) {
+      namesystem.readUnlock();
+    } else {
+      // Expected to have at least one of the locks
+      Preconditions.checkState(false,
+          "Need to be holding either the read or write lock");
+    }
+    // try/finally for retaking FSN lock
+    try {
+      lock.lock();
+      // try/finally for releasing CRM lock
+      try {
+        // If no scan is already ongoing, mark the CRM as dirty and kick
+        if (!isScanning) {
+          needsRescan = true;
+          doRescan.signal();
+        }
+        // Wait until the scan finishes and the count advances
+        final long startCount = scanCount;
+        while (startCount >= scanCount) {
+          try {
+            scanFinished.await();
+          } catch (InterruptedException e) {
+            LOG.warn("Interrupted while waiting for CacheReplicationMonitor"
+                + " rescan", e);
+            break;
+          }
+        }
+      } finally {
+        lock.unlock();
+      }
+    } finally {
+      if (retakeWriteLock) {
+        namesystem.writeLock();
+      } else {
+        namesystem.readLock();
+      }
+    }
+  }
+
+  /**
+   * Indicates to the CacheReplicationMonitor that there have been CacheManager
+   * changes that require a rescan.
    */
-  public synchronized void kick() {
-    rescanImmediately = true;
-    this.notifyAll();
+  public void setNeedsRescan() {
+    lock.lock();
+    try {
+      this.needsRescan = true;
+    } finally {
+      lock.unlock();
+    }
   }
 
   /**
@@ -180,10 +307,14 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
    */
   @Override
   public void close() throws IOException {
-    synchronized(this) {
+    lock.lock();
+    try {
       if (shutdown) return;
       shutdown = true;
-      this.notifyAll();
+      doRescan.signalAll();
+      scanFinished.signalAll();
+    } finally {
+      lock.unlock();
     }
     try {
       if (this.isAlive()) {
@@ -228,12 +359,14 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       // Reset the directive's statistics
       directive.resetStatistics();
       // Skip processing this entry if it has expired
-      LOG.info("Directive expiry is at " + directive.getExpiryTime());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Directive expiry is at " + directive.getExpiryTime());
+      }
       if (directive.getExpiryTime() > 0 && directive.getExpiryTime() <= now) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Skipping directive id " + directive.getId()
               + " because it has expired (" + directive.getExpiryTime() + ">="
-              + now);
+              + now + ")");
         }
         continue;
       }
@@ -280,15 +413,27 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
 
     // Increment the "needed" statistics
     directive.addFilesNeeded(1);
-    long neededTotal = 0;
-    for (BlockInfo blockInfo : blockInfos) {
-      long neededByBlock = 
-          directive.getReplication() * blockInfo.getNumBytes();
-       neededTotal += neededByBlock;
-    }
+    // We don't cache UC blocks, don't add them to the total here
+    long neededTotal = file.computeFileSizeNotIncludingLastUcBlock() *
+        directive.getReplication();
     directive.addBytesNeeded(neededTotal);
 
-    // TODO: Enforce per-pool quotas
+    // The pool's bytesNeeded is incremented as we scan. If the demand
+    // thus far plus the demand of this file would exceed the pool's limit,
+    // do not cache this file.
+    CachePool pool = directive.getPool();
+    if (pool.getBytesNeeded() > pool.getLimit()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("Skipping directive id %d file %s because "
+            + "limit of pool %s would be exceeded (%d > %d)",
+            directive.getId(),
+            file.getFullPathName(),
+            pool.getPoolName(),
+            pool.getBytesNeeded(),
+            pool.getLimit()));
+      }
+      return;
+    }
 
     long cachedTotal = 0;
     for (BlockInfo blockInfo : blockInfos) {

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

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

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

@@ -148,7 +148,7 @@ class BPOfferService {
     return false;
   }
   
-  String getBlockPoolId() {
+  synchronized String getBlockPoolId() {
     if (bpNSInfo != null) {
       return bpNSInfo.getBlockPoolID();
     } else {
@@ -163,7 +163,7 @@ class BPOfferService {
   }
 
   @Override
-  public String toString() {
+  public synchronized String toString() {
     if (bpNSInfo == null) {
       // If we haven't yet connected to our NN, we don't yet know our
       // own block pool ID.

+ 22 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -22,6 +22,7 @@ import static org.apache.hadoop.util.Time.now;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -273,7 +274,8 @@ class BPServiceActor implements Runnable {
   private void reportReceivedDeletedBlocks() throws IOException {
 
     // Generate a list of the pending reports for each storage under the lock
-    Map<String, ReceivedDeletedBlockInfo[]> blockArrays = Maps.newHashMap();
+    ArrayList<StorageReceivedDeletedBlocks> reports =
+        new ArrayList<StorageReceivedDeletedBlocks>(pendingIncrementalBRperStorage.size());
     synchronized (pendingIncrementalBRperStorage) {
       for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
            pendingIncrementalBRperStorage.entrySet()) {
@@ -286,33 +288,34 @@ class BPServiceActor implements Runnable {
           pendingReceivedRequests =
               (pendingReceivedRequests > rdbi.length ?
                   (pendingReceivedRequests - rdbi.length) : 0);
-          blockArrays.put(storageUuid, rdbi);
+          reports.add(new StorageReceivedDeletedBlocks(storageUuid, rdbi));
         }
       }
     }
 
+    if (reports.size() == 0) {
+      // Nothing new to report.
+      return;
+    }
+
     // Send incremental block reports to the Namenode outside the lock
-    for (Map.Entry<String, ReceivedDeletedBlockInfo[]> entry :
-         blockArrays.entrySet()) {
-      final String storageUuid = entry.getKey();
-      final ReceivedDeletedBlockInfo[] rdbi = entry.getValue();
-
-      StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
-          storageUuid, rdbi) };
-      boolean success = false;
-      try {
-        bpNamenode.blockReceivedAndDeleted(bpRegistration,
-            bpos.getBlockPoolId(), report);
-        success = true;
-      } finally {
-        if (!success) {
-          synchronized (pendingIncrementalBRperStorage) {
+    boolean success = false;
+    try {
+      bpNamenode.blockReceivedAndDeleted(bpRegistration,
+          bpos.getBlockPoolId(),
+          reports.toArray(new StorageReceivedDeletedBlocks[reports.size()]));
+      success = true;
+    } finally {
+      if (!success) {
+        synchronized (pendingIncrementalBRperStorage) {
+          for (StorageReceivedDeletedBlocks report : reports) {
             // If we didn't succeed in sending the report, put all of the
             // blocks back onto our queue, but only in the case where we
             // didn't put something newer in the meantime.
             PerStoragePendingIncrementalBR perStorageMap =
-                pendingIncrementalBRperStorage.get(storageUuid);
-            pendingReceivedRequests += perStorageMap.putMissingBlockInfos(rdbi);
+                pendingIncrementalBRperStorage.get(report.getStorageID());
+            pendingReceivedRequests +=
+                perStorageMap.putMissingBlockInfos(report.getBlocks());
           }
         }
       }

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

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

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

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

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

@@ -815,8 +815,6 @@ public class DataNode extends Configured
       storageInfo = new StorageInfo(nsInfo);
     }
 
-    checkDatanodeUuid();
-
     DatanodeID dnId = new DatanodeID(
         streamingAddr.getAddress().getHostAddress(), hostName, 
         storage.getDatanodeUuid(), getXferPort(), getInfoPort(),
@@ -965,6 +963,9 @@ public class DataNode extends Configured
           + ";nsInfo=" + nsInfo + ";dnuuid=" + storage.getDatanodeUuid());
     }
 
+    // If this is a newly formatted DataNode then assign a new DatanodeUuid.
+    checkDatanodeUuid();
+
     synchronized(this)  {
       if (data == null) {
         data = factory.newInstance(this, storage, conf);

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

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

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

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

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

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

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DirectoryWithQuotaFeature.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 /**
  * Quota feature for {@link INodeDirectory}. 
  */
-public final class DirectoryWithQuotaFeature extends INodeDirectory.Feature {
+public final class DirectoryWithQuotaFeature implements INode.Feature {
   public static final long DEFAULT_NAMESPACE_QUOTA = Long.MAX_VALUE;
   public static final long DEFAULT_DISKSPACE_QUOTA = HdfsConstants.QUOTA_RESET;
 
@@ -153,6 +153,10 @@ public final class DirectoryWithQuotaFeature extends INodeDirectory.Feature {
     verifyNamespaceQuota(nsDelta);
     verifyDiskspaceQuota(dsDelta);
   }
+  
+  boolean isQuotaSet() {
+    return nsQuota >= 0 || dsQuota >= 0;
+  }
 
   private String namespaceString() {
     return "namespace: " + (nsQuota < 0? "-": namespace + "/" + nsQuota);

+ 29 - 40
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -67,7 +67,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.Root;
 import org.apache.hadoop.hdfs.util.ByteArray;
@@ -622,8 +621,7 @@ public class FSDirectory implements Closeable {
     // snapshot is taken on the dst tree, changes will be recorded in the latest
     // snapshot of the src tree.
     if (isSrcInSnapshot) {
-      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot(),
-          inodeMap);
+      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot());
       srcIIP.setLastINode(srcChild);
     }
     
@@ -692,11 +690,9 @@ public class FSDirectory implements Closeable {
         }
         // update modification time of dst and the parent of src
         final INode srcParent = srcIIP.getINode(-2);
-        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot(),
-            inodeMap);
+        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot());
         dstParent = dstIIP.getINode(-2); // refresh dstParent
-        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot(),
-            inodeMap);
+        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot());
         // update moved leases with new filename
         getFSNamesystem().unprotectedChangeLease(src, dst);     
 
@@ -734,11 +730,10 @@ public class FSDirectory implements Closeable {
         }
         
         if (isSrcInSnapshot) {
-          // srcParent must be an INodeDirectoryWithSnapshot instance since
-          // isSrcInSnapshot is true and src node has been removed from 
-          // srcParent 
-          ((INodeDirectoryWithSnapshot) srcParent).undoRename4ScrParent(
-              oldSrcChild.asReference(), srcChild, srcIIP.getLatestSnapshot());
+          // srcParent must have snapshot feature since isSrcInSnapshot is true
+          // and src node has been removed from srcParent 
+          srcParent.undoRename4ScrParent(oldSrcChild.asReference(), srcChild,
+              srcIIP.getLatestSnapshot());
         } else {
           // original srcChild is not in latest snapshot, we only need to add
           // the srcChild back
@@ -879,8 +874,7 @@ public class FSDirectory implements Closeable {
     // snapshot is taken on the dst tree, changes will be recorded in the latest
     // snapshot of the src tree.
     if (isSrcInSnapshot) {
-      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot(),
-          inodeMap);
+      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot());
       srcIIP.setLastINode(srcChild);
     }
     
@@ -958,11 +952,9 @@ public class FSDirectory implements Closeable {
         }
 
         final INode srcParent = srcIIP.getINode(-2);
-        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot(),
-            inodeMap);
+        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot());
         dstParent = dstIIP.getINode(-2);
-        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot(),
-            inodeMap);
+        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot());
         // update moved lease with new filename
         getFSNamesystem().unprotectedChangeLease(src, dst);
 
@@ -1019,9 +1011,9 @@ public class FSDirectory implements Closeable {
           withCount.getReferredINode().setLocalName(srcChildName);
         }
         
-        if (srcParent instanceof INodeDirectoryWithSnapshot) {
-          ((INodeDirectoryWithSnapshot) srcParent).undoRename4ScrParent(
-              oldSrcChild.asReference(), srcChild, srcIIP.getLatestSnapshot());
+        if (srcParent.isWithSnapshot()) {
+          srcParent.undoRename4ScrParent(oldSrcChild.asReference(), srcChild,
+              srcIIP.getLatestSnapshot());
         } else {
           // srcParent is not an INodeDirectoryWithSnapshot, we only need to add
           // the srcChild back
@@ -1030,9 +1022,9 @@ public class FSDirectory implements Closeable {
       }
       if (undoRemoveDst) {
         // Rename failed - restore dst
-        if (dstParent instanceof INodeDirectoryWithSnapshot) {
-          ((INodeDirectoryWithSnapshot) dstParent).undoRename4DstParent(
-              removedDst, dstIIP.getLatestSnapshot());
+        if (dstParent.isDirectory() && dstParent.asDirectory().isWithSnapshot()) {
+          dstParent.asDirectory().undoRename4DstParent(removedDst,
+              dstIIP.getLatestSnapshot());
         } else {
           addLastINodeNoQuotaCheck(dstIIP, removedDst);
         }
@@ -1163,8 +1155,7 @@ public class FSDirectory implements Closeable {
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: " + src);
     }
-    inode.setPermission(permissions, inodesInPath.getLatestSnapshot(), 
-        inodeMap);
+    inode.setPermission(permissions, inodesInPath.getLatestSnapshot());
   }
 
   void setOwner(String src, String username, String groupname)
@@ -1189,11 +1180,10 @@ public class FSDirectory implements Closeable {
       throw new FileNotFoundException("File does not exist: " + src);
     }
     if (username != null) {
-      inode = inode.setUser(username, inodesInPath.getLatestSnapshot(),
-          inodeMap);
+      inode = inode.setUser(username, inodesInPath.getLatestSnapshot());
     }
     if (groupname != null) {
-      inode.setGroup(groupname, inodesInPath.getLatestSnapshot(), inodeMap);
+      inode.setGroup(groupname, inodesInPath.getLatestSnapshot());
     }
   }
 
@@ -1266,7 +1256,7 @@ public class FSDirectory implements Closeable {
       if(nodeToRemove == null) continue;
       
       nodeToRemove.setBlocks(null);
-      trgParent.removeChild(nodeToRemove, trgLatestSnapshot, null);
+      trgParent.removeChild(nodeToRemove, trgLatestSnapshot);
       inodeMap.remove(nodeToRemove);
       count++;
     }
@@ -1274,8 +1264,8 @@ public class FSDirectory implements Closeable {
     // update inodeMap
     removeFromInodeMap(Arrays.asList(allSrcInodes));
     
-    trgInode.setModificationTime(timestamp, trgLatestSnapshot, inodeMap);
-    trgParent.updateModificationTime(timestamp, trgLatestSnapshot, inodeMap);
+    trgInode.setModificationTime(timestamp, trgLatestSnapshot);
+    trgParent.updateModificationTime(timestamp, trgLatestSnapshot);
     // update quota on the parent directory ('count' files removed, 0 space)
     unprotectedUpdateCount(trgIIP, trgINodes.length-1, -count, 0);
   }
@@ -1419,7 +1409,7 @@ public class FSDirectory implements Closeable {
 
     // record modification
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
-    targetNode = targetNode.recordModification(latestSnapshot, inodeMap);
+    targetNode = targetNode.recordModification(latestSnapshot);
     iip.setLastINode(targetNode);
 
     // Remove the node from the namespace
@@ -1430,7 +1420,7 @@ public class FSDirectory implements Closeable {
 
     // set the parent's modification time
     final INodeDirectory parent = targetNode.getParent();
-    parent.updateModificationTime(mtime, latestSnapshot, inodeMap);
+    parent.updateModificationTime(mtime, latestSnapshot);
     if (removed == 0) {
       return 0;
     }
@@ -2203,8 +2193,7 @@ public class FSDirectory implements Closeable {
     final INodeDirectory parent = inodes[pos-1].asDirectory();
     boolean added = false;
     try {
-      added = parent.addChild(child, true, iip.getLatestSnapshot(),
-          inodeMap);
+      added = parent.addChild(child, true, iip.getLatestSnapshot());
     } catch (QuotaExceededException e) {
       updateCountNoQuotaCheck(iip, pos,
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
@@ -2242,7 +2231,7 @@ public class FSDirectory implements Closeable {
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
     final INode last = iip.getLastINode();
     final INodeDirectory parent = iip.getINode(-2).asDirectory();
-    if (!parent.removeChild(last, latestSnapshot, inodeMap)) {
+    if (!parent.removeChild(last, latestSnapshot)) {
       return -1;
     }
     INodeDirectory newParent = last.getParent();
@@ -2394,7 +2383,7 @@ public class FSDirectory implements Closeable {
       }
 
       final Snapshot latest = iip.getLatestSnapshot();
-      dirNode = dirNode.recordModification(latest, inodeMap);
+      dirNode = dirNode.recordModification(latest);
       dirNode.setQuota(nsQuota, dsQuota);
       return dirNode;
     }
@@ -2462,7 +2451,7 @@ public class FSDirectory implements Closeable {
     assert hasWriteLock();
     boolean status = false;
     if (mtime != -1) {
-      inode = inode.setModificationTime(mtime, latest, inodeMap);
+      inode = inode.setModificationTime(mtime, latest);
       status = true;
     }
     if (atime != -1) {
@@ -2473,7 +2462,7 @@ public class FSDirectory implements Closeable {
       if (atime <= inodeTime + getFSNamesystem().getAccessTimePrecision() && !force) {
         status =  false;
       } else {
-        inode.setAccessTime(atime, latest, inodeMap);
+        inode.setAccessTime(atime, latest);
         status = true;
       }
     } 

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

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

+ 11 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -24,25 +24,27 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.Arrays;
 import java.util.EnumMap;
+import java.util.EnumSet;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
@@ -55,11 +57,11 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCacheDirectiveInfoOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCacheDirectiveInfoOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
@@ -354,8 +356,8 @@ public class FSEditLogLoader {
       // update the block list.
       
       // Update the salient file attributes.
-      newFile.setAccessTime(addCloseOp.atime, null, fsDir.getINodeMap());
-      newFile.setModificationTime(addCloseOp.mtime, null, fsDir.getINodeMap());
+      newFile.setAccessTime(addCloseOp.atime, null);
+      newFile.setModificationTime(addCloseOp.mtime, null);
       updateBlocks(fsDir, addCloseOp, newFile);
       break;
     }
@@ -373,8 +375,8 @@ public class FSEditLogLoader {
       final INodeFile file = INodeFile.valueOf(iip.getINode(0), addCloseOp.path);
 
       // Update the salient file attributes.
-      file.setAccessTime(addCloseOp.atime, null, fsDir.getINodeMap());
-      file.setModificationTime(addCloseOp.mtime, null, fsDir.getINodeMap());
+      file.setAccessTime(addCloseOp.atime, null);
+      file.setModificationTime(addCloseOp.mtime, null);
       updateBlocks(fsDir, addCloseOp, file);
 
       // Now close the file
@@ -650,7 +652,7 @@ public class FSEditLogLoader {
       ModifyCacheDirectiveInfoOp modifyOp =
           (ModifyCacheDirectiveInfoOp) op;
       fsNamesys.getCacheManager().modifyDirective(
-          modifyOp.directive, null);
+          modifyOp.directive, null, EnumSet.of(CacheFlag.FORCE));
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
       }

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

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

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

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

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -52,9 +52,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -731,9 +731,10 @@ public class FSImageFormat {
       if (nsQuota >= 0 || dsQuota >= 0) {
         dir.addDirectoryWithQuotaFeature(nsQuota, dsQuota);
       }
-      return snapshottable ? new INodeDirectorySnapshottable(dir)
-          : withSnapshot ? new INodeDirectoryWithSnapshot(dir)
-          : dir;
+      if (withSnapshot) {
+        dir.addSnapshotFeature(null);
+      }
+      return snapshottable ? new INodeDirectorySnapshottable(dir) : dir;
     } else if (numBlocks == -2) {
       //symlink
 
@@ -1113,10 +1114,10 @@ public class FSImageFormat {
       final ReadOnlyList<INode> children = current.getChildrenList(null);
       int dirNum = 0;
       List<INodeDirectory> snapshotDirs = null;
-      if (current instanceof INodeDirectoryWithSnapshot) {
+      DirectoryWithSnapshotFeature sf = current.getDirectoryWithSnapshotFeature();
+      if (sf != null) {
         snapshotDirs = new ArrayList<INodeDirectory>();
-        ((INodeDirectoryWithSnapshot) current).getSnapshotDirectory(
-            snapshotDirs);
+        sf.getSnapshotDirectory(snapshotDirs);
         dirNum += snapshotDirs.size();
       }
       

+ 206 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

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

+ 14 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -126,6 +126,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
@@ -735,12 +736,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       this.dtSecretManager = createDelegationTokenSecretManager(conf);
       this.dir = new FSDirectory(fsImage, this, conf);
       this.snapshotManager = new SnapshotManager(dir);
-      writeLock();
-      try {
-        this.cacheManager = new CacheManager(this, conf, blockManager);
-      } finally {
-        writeUnlock();
-      }
+      this.cacheManager = new CacheManager(this, conf, blockManager);
       this.safeMode = new SafeModeInfo(conf);
       this.auditLoggers = initAuditLoggers(conf);
       this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
@@ -2295,7 +2291,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
       boolean writeToEditLog, Snapshot latestSnapshot, boolean logRetryCache)
       throws IOException {
-    file = file.recordModification(latestSnapshot, dir.getINodeMap());
+    file = file.recordModification(latestSnapshot);
     final INodeFile cons = file.toUnderConstruction(leaseHolder, clientMachine,
         clientNode);
 
@@ -2877,8 +2873,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       writeUnlock();
     }
     getEditLog().logSync();
-    NameNode.stateChangeLog.info("DIR* completeFile: " + src + " is closed by "
-        + holder);
+    if (success) {
+      NameNode.stateChangeLog.info("DIR* completeFile: " + src
+          + " is closed by " + holder);
+    }
     return success;
   }
 
@@ -3783,8 +3781,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     Preconditions.checkArgument(uc != null);
     leaseManager.removeLease(uc.getClientName(), src);
     
-    pendingFile = pendingFile.recordModification(latestSnapshot,
-        dir.getINodeMap());
+    pendingFile = pendingFile.recordModification(latestSnapshot);
 
     // The file is no longer pending.
     // Create permanent INode, update blocks. No need to replace the inode here
@@ -7056,8 +7053,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  long addCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  long addCacheDirective(CacheDirectiveInfo directive, EnumSet<CacheFlag> flags)
+      throws IOException {
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = isPermissionEnabled ?
         getPermissionChecker() : null;
@@ -7080,7 +7077,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             "for this operation.");
       }
       CacheDirectiveInfo effectiveDirective = 
-          cacheManager.addDirective(directive, pc);
+          cacheManager.addDirective(directive, pc, flags);
       getEditLog().logAddCacheDirectiveInfo(effectiveDirective,
           cacheEntry != null);
       result = effectiveDirective.getId();
@@ -7098,8 +7095,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return result;
   }
 
-  void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
+  void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = isPermissionEnabled ?
         getPermissionChecker() : null;
@@ -7115,7 +7112,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new SafeModeException(
             "Cannot add cache directive", safeMode);
       }
-      cacheManager.modifyDirective(directive, pc);
+      cacheManager.modifyDirective(directive, pc, flags);
       getEditLog().logModifyCacheDirectiveInfo(directive,
           cacheEntry != null);
       success = true;

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

@@ -29,7 +29,7 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
  * Feature for under-construction file.
  */
 @InterfaceAudience.Private
-public class FileUnderConstructionFeature extends INodeFile.Feature {
+public class FileUnderConstructionFeature implements INode.Feature {
   private String clientName; // lease holder
   private final String clientMachine;
   // if client is a cluster node too.

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

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

+ 22 - 65
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.Diff;
@@ -96,9 +95,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setUser(String user);
 
   /** Set user */
-  final INode setUser(String user, Snapshot latest, INodeMap inodeMap)
+  final INode setUser(String user, Snapshot latest)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setUser(user);
     return nodeToUpdate;
   }
@@ -120,9 +119,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setGroup(String group);
 
   /** Set group */
-  final INode setGroup(String group, Snapshot latest, INodeMap inodeMap)
+  final INode setGroup(String group, Snapshot latest)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setGroup(group);
     return nodeToUpdate;
   }
@@ -145,9 +144,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setPermission(FsPermission permission);
 
   /** Set the {@link FsPermission} of this {@link INode} */
-  INode setPermission(FsPermission permission, Snapshot latest,
-      INodeMap inodeMap) throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+  INode setPermission(FsPermission permission, Snapshot latest) 
+      throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setPermission(permission);
     return nodeToUpdate;
   }
@@ -231,14 +230,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *
    * @param latest the latest snapshot that has been taken.
    *        Note that it is null if no snapshots have been taken.
-   * @param inodeMap while recording modification, the inode or its parent may 
-   *                 get replaced, and the inodeMap needs to be updated.
    * @return The current inode, which usually is the same object of this inode.
    *         However, in some cases, this inode may be replaced with a new inode
    *         for maintaining snapshots. The current inode is then the new inode.
    */
-  abstract INode recordModification(final Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException;
+  abstract INode recordModification(final Snapshot latest)
+      throws QuotaExceededException;
 
   /** Check whether it's a reference. */
   public boolean isReference() {
@@ -318,7 +315,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * Call recordModification(..) to capture the current states.
    * Mark the INode as deleted.
    * 
-   * 1.4 The current inode is a {@link INodeDirectoryWithSnapshot}.
+   * 1.4 The current inode is an {@link INodeDirectory} with snapshot feature.
    * Call recordModification(..) to capture the current states. 
    * Destroy files/directories created after the latest snapshot 
    * (i.e., the inodes stored in the created list of the latest snapshot).
@@ -329,7 +326,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * 2.2 To clean {@link INodeDirectory}: recursively clean its children.
    * 2.3 To clean INodeFile with snapshot: delete the corresponding snapshot in
    * its diff list.
-   * 2.4 To clean {@link INodeDirectoryWithSnapshot}: delete the corresponding 
+   * 2.4 To clean {@link INodeDirectory} with snapshot: delete the corresponding 
    * snapshot in its diff list. Recursively clean its children.
    * </pre>
    * 
@@ -575,16 +572,16 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   }
 
   /** Update modification time if it is larger than the current value. */
-  public abstract INode updateModificationTime(long mtime, Snapshot latest,
-      INodeMap inodeMap) throws QuotaExceededException;
+  public abstract INode updateModificationTime(long mtime, Snapshot latest) 
+      throws QuotaExceededException;
 
   /** Set the last modification time of inode. */
   public abstract void setModificationTime(long modificationTime);
 
   /** Set the last modification time of inode. */
   public final INode setModificationTime(long modificationTime,
-      Snapshot latest, INodeMap inodeMap) throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+      Snapshot latest) throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setModificationTime(modificationTime);
     return nodeToUpdate;
   }
@@ -611,9 +608,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   /**
    * Set last access time of inode.
    */
-  public final INode setAccessTime(long accessTime, Snapshot latest,
-      INodeMap inodeMap) throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+  public final INode setAccessTime(long accessTime, Snapshot latest)
+      throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setAccessTime(accessTime);
     return nodeToUpdate;
   }
@@ -753,50 +750,10 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     }
   }
 
-  /** INode feature such as {@link FileUnderConstructionFeature}
-   *  and {@link DirectoryWithQuotaFeature}.
+  /** 
+   * INode feature such as {@link FileUnderConstructionFeature}
+   * and {@link DirectoryWithQuotaFeature}.
    */
-  interface Feature<F extends Feature<F>> {
-    /** @return the next feature. */
-    public F getNextFeature();
-
-    /** Set the next feature. */
-    public void setNextFeature(F next);
-
-    /** Utility methods such as addFeature and removeFeature. */
-    static class Util {
-      /**
-       * Add a feature to the linked list.
-       * @return the new head.
-       */
-      static <F extends Feature<F>> F addFeature(F feature, F head) {
-        feature.setNextFeature(head);
-        return feature;
-      }
-
-      /**
-       * Remove a feature from the linked list.
-       * @return the new head.
-       */
-      static <F extends Feature<F>> F removeFeature(F feature, F head) {
-        if (feature == head) {
-          final F newHead = head.getNextFeature();
-          head.setNextFeature(null);
-          return newHead;
-        } else if (head != null) {
-          F prev = head;
-          F curr = head.getNextFeature();
-          for (; curr != null && curr != feature;
-              prev = curr, curr = curr.getNextFeature())
-            ;
-          if (curr != null) {
-            prev.setNextFeature(curr.getNextFeature());
-            curr.setNextFeature(null);
-            return head;
-          }
-        }
-        throw new IllegalStateException("Feature " + feature + " not found.");
-      }
-    }
+  public interface Feature {
   }
 }

+ 265 - 126
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -32,9 +32,11 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -45,20 +47,6 @@ import com.google.common.base.Preconditions;
  */
 public class INodeDirectory extends INodeWithAdditionalFields
     implements INodeDirectoryAttributes {
-  /** Directory related features such as quota and snapshots. */
-  public static abstract class Feature implements INode.Feature<Feature> {
-    private Feature nextFeature;
-
-    @Override
-    public Feature getNextFeature() {
-      return nextFeature;
-    }
-
-    @Override
-    public void setNextFeature(Feature next) {
-      this.nextFeature = next;
-    }
-  }
 
   /** Cast INode to INodeDirectory. */
   public static INodeDirectory valueOf(INode inode, Object path
@@ -78,9 +66,6 @@ public class INodeDirectory extends INodeWithAdditionalFields
 
   private List<INode> children = null;
   
-  /** A linked list of {@link Feature}s. */
-  private Feature headFeature = null;
-
   /** constructor */
   public INodeDirectory(long id, byte[] name, PermissionStatus permissions,
       long mtime) {
@@ -102,7 +87,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
       }
     }
     if (copyFeatures) {
-      this.headFeature = other.headFeature;
+      this.features = other.features;
     }
   }
 
@@ -160,7 +145,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * otherwise, return null.
    */
   public final DirectoryWithQuotaFeature getDirectoryWithQuotaFeature() {
-    for(Feature f = headFeature; f != null; f = f.nextFeature) {
+    for (Feature f : features) {
       if (f instanceof DirectoryWithQuotaFeature) {
         return (DirectoryWithQuotaFeature)f;
       }
@@ -182,51 +167,52 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return quota;
   }
 
-  private void addFeature(Feature f) {
-    headFeature = INode.Feature.Util.addFeature(f, headFeature);
-  }
-
-  private void removeFeature(Feature f) {
-    headFeature = INode.Feature.Util.removeFeature(f, headFeature);
-  }
-
   private int searchChildren(byte[] name) {
     return children == null? -1: Collections.binarySearch(children, name);
   }
-
+  
+  protected DirectoryWithSnapshotFeature addSnapshotFeature(
+      DirectoryDiffList diffs) {
+    Preconditions.checkState(!isWithSnapshot(), 
+        "Directory is already with snapshot");
+    DirectoryWithSnapshotFeature sf = new DirectoryWithSnapshotFeature(diffs);
+    addFeature(sf);
+    return sf;
+  }
+  
   /**
-   * Remove the specified child from this directory.
-   * 
-   * @param child the child inode to be removed
-   * @param latest See {@link INode#recordModification(Snapshot, INodeMap)}.
+   * If feature list contains a {@link DirectoryWithSnapshotFeature}, return it;
+   * otherwise, return null.
    */
-  public boolean removeChild(INode child, Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    if (isInLatestSnapshot(latest)) {
-      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
-          .removeChild(child, latest, inodeMap);
+  public final DirectoryWithSnapshotFeature getDirectoryWithSnapshotFeature() {
+    for (Feature f : features) {
+      if (f instanceof DirectoryWithSnapshotFeature) {
+        return (DirectoryWithSnapshotFeature) f;
+      }
     }
-
-    return removeChild(child);
+    return null;
   }
 
-  /** 
-   * Remove the specified child from this directory.
-   * The basic remove method which actually calls children.remove(..).
-   *
-   * @param child the child inode to be removed
-   * 
-   * @return true if the child is removed; false if the child is not found.
-   */
-  protected final boolean removeChild(final INode child) {
-    final int i = searchChildren(child.getLocalNameBytes());
-    if (i < 0) {
-      return false;
-    }
-
-    final INode removed = children.remove(i);
-    Preconditions.checkState(removed == child);
-    return true;
+  /** Is this file has the snapshot feature? */
+  public final boolean isWithSnapshot() {
+    return getDirectoryWithSnapshotFeature() != null;
+  }
+  
+  public DirectoryDiffList getDiffs() {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    return sf != null ? sf.getDiffs() : null;
+  }
+  
+  @Override
+  public INodeDirectoryAttributes getSnapshotINode(Snapshot snapshot) {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    return sf == null ? this : sf.getDiffs().getSnapshotINode(snapshot, this);
+  }
+  
+  @Override
+  public String toDetailString() {
+    DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
+    return super.toDetailString() + (sf == null ? "" : ", " + sf.getDiffs()); 
   }
 
   /** Replace itself with an {@link INodeDirectorySnapshottable}. */
@@ -235,16 +221,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
     Preconditions.checkState(!(this instanceof INodeDirectorySnapshottable),
         "this is already an INodeDirectorySnapshottable, this=%s", this);
     final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(this);
-    replaceSelf(s, inodeMap).saveSelf2Snapshot(latest, this);
+    replaceSelf(s, inodeMap).getDirectoryWithSnapshotFeature().getDiffs()
+        .saveSelf2Snapshot(latest, s, this);
     return s;
   }
 
-  /** Replace itself with an {@link INodeDirectoryWithSnapshot}. */
-  public INodeDirectoryWithSnapshot replaceSelf4INodeDirectoryWithSnapshot(
-      final INodeMap inodeMap) {
-    return replaceSelf(new INodeDirectoryWithSnapshot(this), inodeMap);
-  }
-
   /** Replace itself with {@link INodeDirectory}. */
   public INodeDirectory replaceSelf4INodeDirectory(final INodeMap inodeMap) {
     Preconditions.checkState(getClass() != INodeDirectory.class,
@@ -270,7 +251,13 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return newDir;
   }
   
-  /** Replace the given child with a new child. */
+  /** 
+   * Replace the given child with a new child. Note that we no longer need to
+   * replace an normal INodeDirectory or INodeFile into an
+   * INodeDirectoryWithSnapshot or INodeFileUnderConstruction. The only cases
+   * for child replacement is for {@link INodeDirectorySnapshottable} and 
+   * reference nodes.
+   */
   public void replaceChild(INode oldChild, final INode newChild,
       final INodeMap inodeMap) {
     Preconditions.checkNotNull(children);
@@ -281,24 +268,24 @@ public class INodeDirectory extends INodeWithAdditionalFields
             .asReference().getReferredINode());
     oldChild = children.get(i);
     
-    if (oldChild.isReference() && !newChild.isReference()) {
-      // replace the referred inode, e.g., 
-      // INodeFileWithSnapshot -> INodeFileUnderConstructionWithSnapshot
-      final INode withCount = oldChild.asReference().getReferredINode();
-      withCount.asReference().setReferredINode(newChild);
-    } else {
-      if (oldChild.isReference()) {
-        // both are reference nodes, e.g., DstReference -> WithName
-        final INodeReference.WithCount withCount = 
-            (WithCount) oldChild.asReference().getReferredINode();
-        withCount.removeReference(oldChild.asReference());
-      }
-      children.set(i, newChild);
+    if (oldChild.isReference() && newChild.isReference()) {
+      // both are reference nodes, e.g., DstReference -> WithName
+      final INodeReference.WithCount withCount = 
+          (WithCount) oldChild.asReference().getReferredINode();
+      withCount.removeReference(oldChild.asReference());
+    }
+    children.set(i, newChild);
+    
+    // replace the instance in the created list of the diff list
+    DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
     }
+    
     // update the inodeMap
     if (inodeMap != null) {
       inodeMap.put(newChild);
-    }
+    }    
   }
 
   INodeReference.WithName replaceChild4ReferenceWithName(INode oldChild,
@@ -323,14 +310,18 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   @Override
-  public INodeDirectory recordModification(Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    if (isInLatestSnapshot(latest)) {
-      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
-          .recordModification(latest, inodeMap);
-    } else {
-      return this;
+  public INodeDirectory recordModification(Snapshot latest) 
+      throws QuotaExceededException {
+    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
+      // add snapshot feature if necessary
+      DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+      if (sf == null) {
+        sf = addSnapshotFeature(null);
+      }
+      // record self in the diff list if necessary
+      sf.getDiffs().saveSelf2Snapshot(latest, this, null);
     }
+    return this;
   }
 
   /**
@@ -339,13 +330,17 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * @return the child inode, which may be replaced.
    */
   public INode saveChild2Snapshot(final INode child, final Snapshot latest,
-      final INode snapshotCopy, final INodeMap inodeMap)
-      throws QuotaExceededException {
+      final INode snapshotCopy) throws QuotaExceededException {
     if (latest == null) {
       return child;
     }
-    return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
-        .saveChild2Snapshot(child, latest, snapshotCopy, inodeMap);
+    
+    // add snapshot feature if necessary
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf == null) {
+      sf = this.addSnapshotFeature(null);
+    }
+    return sf.saveChild2Snapshot(this, child, latest, snapshotCopy);
   }
 
   /**
@@ -356,9 +351,36 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * @return the child inode.
    */
   public INode getChild(byte[] name, Snapshot snapshot) {
-    final ReadOnlyList<INode> c = getChildrenList(snapshot);
-    final int i = ReadOnlyList.Util.binarySearch(c, name);
-    return i < 0? null: c.get(i);
+    DirectoryWithSnapshotFeature sf;
+    if (snapshot == null || (sf = getDirectoryWithSnapshotFeature()) == null) {
+      ReadOnlyList<INode> c = getCurrentChildrenList();
+      final int i = ReadOnlyList.Util.binarySearch(c, name);
+      return i < 0 ? null : c.get(i);
+    }
+    
+    return sf.getChild(this, name, snapshot);
+  }
+  
+  /**
+   * @param snapshot
+   *          if it is not null, get the result from the given snapshot;
+   *          otherwise, get the result from the current directory.
+   * @return the current children list if the specified snapshot is null;
+   *         otherwise, return the children list corresponding to the snapshot.
+   *         Note that the returned list is never null.
+   */
+  public ReadOnlyList<INode> getChildrenList(final Snapshot snapshot) {
+    DirectoryWithSnapshotFeature sf;
+    if (snapshot == null
+        || (sf = this.getDirectoryWithSnapshotFeature()) == null) {
+      return getCurrentChildrenList();
+    }
+    return sf.getChildrenList(this, snapshot);
+  }
+  
+  private ReadOnlyList<INode> getCurrentChildrenList() {
+    return children == null ? ReadOnlyList.Util.<INode> emptyList()
+        : ReadOnlyList.Util.asReadOnlyList(children);
   }
 
   /** @return the {@link INodesInPath} containing only the last inode. */
@@ -424,6 +446,41 @@ public class INodeDirectory extends INodeWithAdditionalFields
     }
     return -nextPos;
   }
+  
+  /**
+   * Remove the specified child from this directory.
+   */
+  public boolean removeChild(INode child, Snapshot latest)
+      throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      // create snapshot feature if necessary
+      DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
+      if (sf == null) {
+        sf = this.addSnapshotFeature(null);
+      }
+      return sf.removeChild(this, child, latest);
+    }
+    return removeChild(child);
+  }
+  
+  /** 
+   * Remove the specified child from this directory.
+   * The basic remove method which actually calls children.remove(..).
+   *
+   * @param child the child inode to be removed
+   * 
+   * @return true if the child is removed; false if the child is not found.
+   */
+  public boolean removeChild(final INode child) {
+    final int i = searchChildren(child.getLocalNameBytes());
+    if (i < 0) {
+      return false;
+    }
+
+    final INode removed = children.remove(i);
+    Preconditions.checkState(removed == child);
+    return true;
+  }
 
   /**
    * Add a child inode to the directory.
@@ -432,34 +489,32 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * @param setModTime set modification time for the parent node
    *                   not needed when replaying the addition and 
    *                   the parent already has the proper mod time
-   * @param inodeMap update the inodeMap if the directory node gets replaced
    * @return false if the child with this name already exists; 
    *         otherwise, return true;
    */
   public boolean addChild(INode node, final boolean setModTime,
-      final Snapshot latest, final INodeMap inodeMap)
-      throws QuotaExceededException {
+      final Snapshot latest) throws QuotaExceededException {
     final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
       return false;
     }
 
     if (isInLatestSnapshot(latest)) {
-      INodeDirectoryWithSnapshot sdir = 
-          replaceSelf4INodeDirectoryWithSnapshot(inodeMap);
-      boolean added = sdir.addChild(node, setModTime, latest, inodeMap);
-      return added;
+      // create snapshot feature if necessary
+      DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
+      if (sf == null) {
+        sf = this.addSnapshotFeature(null);
+      }
+      return sf.addChild(this, node, setModTime, latest);
     }
     addChild(node, low);
     if (setModTime) {
       // update modification time of the parent directory
-      updateModificationTime(node.getModificationTime(), latest, inodeMap);
+      updateModificationTime(node.getModificationTime(), latest);
     }
     return true;
   }
 
-
-  /** The same as addChild(node, false, null, false) */
   public boolean addChild(INode node) {
     final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
@@ -488,21 +543,34 @@ public class INodeDirectory extends INodeWithAdditionalFields
   @Override
   public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
       int lastSnapshotId) {
-    final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
-    if (q != null) {
-      if (useCache && isQuotaSet()) {
-        q.addNamespaceDiskspace(counts);
-      } else {
-        computeDirectoryQuotaUsage(counts, false, lastSnapshotId);
+    final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    
+    // we are computing the quota usage for a specific snapshot here, i.e., the
+    // computation only includes files/directories that exist at the time of the
+    // given snapshot
+    if (sf != null && lastSnapshotId != Snapshot.INVALID_ID
+        && !(useCache && isQuotaSet())) {
+      Snapshot lastSnapshot = sf.getDiffs().getSnapshotById(lastSnapshotId);
+      ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshot);
+      for (INode child : childrenList) {
+        child.computeQuotaUsage(counts, useCache, lastSnapshotId);
       }
+      counts.add(Quota.NAMESPACE, 1);
       return counts;
+    }
+    
+    // compute the quota usage in the scope of the current directory tree
+    final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
+    if (useCache && q != null && q.isQuotaSet()) { // use the cached quota
+      return q.addNamespaceDiskspace(counts);
     } else {
+      useCache = q != null && !q.isQuotaSet() ? false : useCache;
       return computeDirectoryQuotaUsage(counts, useCache, lastSnapshotId);
     }
   }
 
-  Quota.Counts computeDirectoryQuotaUsage(Quota.Counts counts, boolean useCache,
-      int lastSnapshotId) {
+  private Quota.Counts computeDirectoryQuotaUsage(Quota.Counts counts,
+      boolean useCache, int lastSnapshotId) {
     if (children != null) {
       for (INode child : children) {
         child.computeQuotaUsage(counts, useCache, lastSnapshotId);
@@ -514,12 +582,21 @@ public class INodeDirectory extends INodeWithAdditionalFields
   /** Add quota usage for this inode excluding children. */
   public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
     counts.add(Quota.NAMESPACE, 1);
+    // include the diff list
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.computeQuotaUsage4CurrentDirectory(counts);
+    }
     return counts;
   }
 
   @Override
   public ContentSummaryComputationContext computeContentSummary(
       ContentSummaryComputationContext summary) {
+    final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.computeContentSummary4Snapshot(summary.getCounts());
+    }
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
     if (q != null) {
       return q.computeContentSummary(this, summary);
@@ -546,13 +623,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
       if (lastYieldCount == summary.getYieldCount()) {
         continue;
       }
-
       // The locks were released and reacquired. Check parent first.
       if (getParent() == null) {
         // Stop further counting and return whatever we have so far.
         break;
       }
-
       // Obtain the children list again since it may have been modified.
       childrenList = getChildrenList(null);
       // Reposition in case the children list is changed. Decrement by 1
@@ -562,24 +637,77 @@ public class INodeDirectory extends INodeWithAdditionalFields
 
     // Increment the directory count for this directory.
     summary.getCounts().add(Content.DIRECTORY, 1);
-
     // Relinquish and reacquire locks if necessary.
     summary.yield();
-
     return summary;
   }
-
+  
   /**
-   * @param snapshot
-   *          if it is not null, get the result from the given snapshot;
-   *          otherwise, get the result from the current directory.
-   * @return the current children list if the specified snapshot is null;
-   *         otherwise, return the children list corresponding to the snapshot.
-   *         Note that the returned list is never null.
+   * This method is usually called by the undo section of rename.
+   * 
+   * Before calling this function, in the rename operation, we replace the
+   * original src node (of the rename operation) with a reference node (WithName
+   * instance) in both the children list and a created list, delete the
+   * reference node from the children list, and add it to the corresponding
+   * deleted list.
+   * 
+   * To undo the above operations, we have the following steps in particular:
+   * 
+   * <pre>
+   * 1) remove the WithName node from the deleted list (if it exists) 
+   * 2) replace the WithName node in the created list with srcChild 
+   * 3) add srcChild back as a child of srcParent. Note that we already add 
+   * the node into the created list of a snapshot diff in step 2, we do not need
+   * to add srcChild to the created list of the latest snapshot.
+   * </pre>
+   * 
+   * We do not need to update quota usage because the old child is in the 
+   * deleted list before. 
+   * 
+   * @param oldChild
+   *          The reference node to be removed/replaced
+   * @param newChild
+   *          The node to be added back
+   * @param latestSnapshot
+   *          The latest snapshot. Note this may not be the last snapshot in the
+   *          diff list, since the src tree of the current rename operation
+   *          may be the dst tree of a previous rename.
+   * @throws QuotaExceededException should not throw this exception
    */
-  public ReadOnlyList<INode> getChildrenList(final Snapshot snapshot) {
-    return children == null ? ReadOnlyList.Util.<INode>emptyList()
-        : ReadOnlyList.Util.asReadOnlyList(children);
+  public void undoRename4ScrParent(final INodeReference oldChild,
+      final INode newChild, Snapshot latestSnapshot)
+      throws QuotaExceededException {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    Preconditions.checkState(sf != null,
+        "Directory does not have snapshot feature");
+    sf.getDiffs().removeChild(ListType.DELETED, oldChild);
+    sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
+    addChild(newChild, true, null);
+  }
+  
+  /**
+   * Undo the rename operation for the dst tree, i.e., if the rename operation
+   * (with OVERWRITE option) removes a file/dir from the dst tree, add it back
+   * and delete possible record in the deleted list.  
+   */
+  public void undoRename4DstParent(final INode deletedChild,
+      Snapshot latestSnapshot) throws QuotaExceededException {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    Preconditions.checkState(sf != null,
+        "Directory does not have snapshot feature");
+    boolean removeDeletedChild = sf.getDiffs().removeChild(ListType.DELETED,
+        deletedChild);
+    // pass null for inodeMap since the parent node will not get replaced when
+    // undoing rename
+    final boolean added = addChild(deletedChild, true, removeDeletedChild ? null
+        : latestSnapshot);
+    // update quota usage if adding is successfully and the old child has not
+    // been stored in deleted list before
+    if (added && !removeDeletedChild) {
+      final Quota.Counts counts = deletedChild.computeQuotaUsage();
+      addSpaceConsumed(counts.get(Quota.NAMESPACE),
+          counts.get(Quota.DISKSPACE), false);
+    }
   }
 
   /** Set the children list to null. */
@@ -603,7 +731,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
     // the diff list, the snapshot to be deleted has been combined or renamed
     // to its latest previous snapshot. (besides, we also need to consider nodes
     // created after prior but before snapshot. this will be done in 
-    // INodeDirectoryWithSnapshot#cleanSubtree)
+    // DirectoryWithSnapshotFeature)
     Snapshot s = snapshot != null && prior != null ? prior : snapshot;
     for (INode child : getChildrenList(s)) {
       if (snapshot != null && excludedNodes != null
@@ -621,6 +749,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
   @Override
   public void destroyAndCollectBlocks(final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
+    final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.clear(this, collectedBlocks, removedINodes);
+    }
     for (INode child : getChildrenList(null)) {
       child.destroyAndCollectBlocks(collectedBlocks, removedINodes);
     }
@@ -633,6 +765,13 @@ public class INodeDirectory extends INodeWithAdditionalFields
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes, final boolean countDiffChange)
       throws QuotaExceededException {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    // there is snapshot data
+    if (sf != null) {
+      return sf.cleanDirectory(this, snapshot, prior, collectedBlocks,
+          removedINodes, countDiffChange);
+    }
+    // there is no snapshot data
     if (prior == null && snapshot == null) {
       // destroy the whole subtree and collect blocks that should be deleted
       Quota.Counts counts = Quota.Counts.newInstance();

+ 16 - 39
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -27,7 +27,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.server.blockmanagement.*;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
@@ -41,23 +45,6 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 public class INodeFile extends INodeWithAdditionalFields
     implements INodeFileAttributes, BlockCollection {
-  /**
-   * A feature contains specific information for a type of INodeFile. E.g.,
-   * we can have separate features for Under-Construction and Snapshot.
-   */
-  public static abstract class Feature implements INode.Feature<Feature> {
-    private Feature nextFeature;
-
-    @Override
-    public Feature getNextFeature() {
-      return nextFeature;
-    }
-
-    @Override
-    public void setNextFeature(Feature next) {
-      this.nextFeature = next;
-    }
-  }
 
   /** The same as valueOf(inode, path, false). */
   public static INodeFile valueOf(INode inode, String path
@@ -120,8 +107,6 @@ public class INodeFile extends INodeWithAdditionalFields
 
   private BlockInfo[] blocks;
 
-  private Feature headFeature;
-
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
       long preferredBlockSize) {
@@ -135,7 +120,7 @@ public class INodeFile extends INodeWithAdditionalFields
     super(that);
     this.header = that.header;
     this.blocks = that.blocks;
-    this.headFeature = that.headFeature;
+    this.features = that.features;
   }
   
   public INodeFile(INodeFile that, FileDiffList diffs) {
@@ -144,14 +129,6 @@ public class INodeFile extends INodeWithAdditionalFields
     this.addSnapshotFeature(diffs);
   }
 
-  private void addFeature(Feature f) {
-    headFeature = INode.Feature.Util.addFeature(f, headFeature);
-  }
-
-  private void removeFeature(Feature f) {
-    headFeature = INode.Feature.Util.removeFeature(f, headFeature);
-  }
-
   /** @return true unconditionally. */
   @Override
   public final boolean isFile() {
@@ -171,7 +148,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * otherwise, return null.
    */
   public final FileUnderConstructionFeature getFileUnderConstructionFeature() {
-    for (Feature f = this.headFeature; f != null; f = f.nextFeature) {
+    for (Feature f : features) {
       if (f instanceof FileUnderConstructionFeature) {
         return (FileUnderConstructionFeature) f;
       }
@@ -273,6 +250,8 @@ public class INodeFile extends INodeWithAdditionalFields
   /* Start of Snapshot Feature */
 
   private FileWithSnapshotFeature addSnapshotFeature(FileDiffList diffs) {
+    Preconditions.checkState(!isWithSnapshot(), 
+        "File is already with snapshot");
     FileWithSnapshotFeature sf = new FileWithSnapshotFeature(diffs);
     this.addFeature(sf);
     return sf;
@@ -283,7 +262,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * otherwise, return null.
    */
   public final FileWithSnapshotFeature getFileWithSnapshotFeature() {
-    for (Feature f = headFeature; f != null; f = f.nextFeature) {
+    for (Feature f: features) {
       if (f instanceof FileWithSnapshotFeature) {
         return (FileWithSnapshotFeature) f;
       }
@@ -306,25 +285,23 @@ public class INodeFile extends INodeWithAdditionalFields
   public INodeFileAttributes getSnapshotINode(final Snapshot snapshot) {
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
     if (sf != null) {
-      return sf.getSnapshotINode(this, snapshot);
+      return sf.getDiffs().getSnapshotINode(snapshot, this);
     } else {
       return this;
     }
   }
 
   @Override
-  public INodeFile recordModification(final Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    if (isInLatestSnapshot(latest)) {
+  public INodeFile recordModification(final Snapshot latest) 
+      throws QuotaExceededException {
+    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
       // the file is in snapshot, create a snapshot feature if it does not have
       FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
       if (sf == null) {
         sf = addSnapshotFeature(null);
       }
       // record self in the diff list if necessary
-      if (!shouldRecordInSrcSnapshot(latest)) {
-        sf.getDiffs().saveSelf2Snapshot(latest, this, null);
-      }
+      sf.getDiffs().saveSelf2Snapshot(latest, this, null);
     }
     return this;
   }
@@ -376,7 +353,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication, Snapshot latest,
       final INodeMap inodeMap) throws QuotaExceededException {
-    final INodeFile nodeToUpdate = recordModification(latest, inodeMap);
+    final INodeFile nodeToUpdate = recordModification(latest);
     nodeToUpdate.setFileReplication(replication);
     return nodeToUpdate;
   }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java

@@ -89,8 +89,7 @@ public class INodeMap {
         "", "", new FsPermission((short) 0)), 0, 0) {
       
       @Override
-      INode recordModification(Snapshot latest, INodeMap inodeMap)
-          throws QuotaExceededException {
+      INode recordModification(Snapshot latest) throws QuotaExceededException {
         return null;
       }
       

+ 33 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.base.Preconditions;
@@ -103,9 +103,12 @@ public abstract class INodeReference extends INode {
       INode referred = wc.getReferredINode();
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
         return referred.asFile().getDiffs().getPrior(wn.lastSnapshotId);
-      } else if (referred instanceof INodeDirectoryWithSnapshot) { 
-        return ((INodeDirectoryWithSnapshot) referred).getDiffs().getPrior(
-            wn.lastSnapshotId);
+      } else if (referred.isDirectory()) {
+        DirectoryWithSnapshotFeature sf = referred.asDirectory()
+            .getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          return sf.getDiffs().getPrior(wn.lastSnapshotId);
+        }
       }
     }
     return null;
@@ -231,9 +234,9 @@ public abstract class INodeReference extends INode {
   }
   
   @Override
-  public final INode updateModificationTime(long mtime, Snapshot latest,
-      INodeMap inodeMap) throws QuotaExceededException {
-    return referred.updateModificationTime(mtime, latest, inodeMap);
+  public final INode updateModificationTime(long mtime, Snapshot latest) 
+      throws QuotaExceededException {
+    return referred.updateModificationTime(mtime, latest);
   }
   
   @Override
@@ -252,9 +255,9 @@ public abstract class INodeReference extends INode {
   }
 
   @Override
-  final INode recordModification(Snapshot latest, final INodeMap inodeMap)
+  final INode recordModification(Snapshot latest)
       throws QuotaExceededException {
-    referred.recordModification(latest, inodeMap);
+    referred.recordModification(latest);
     // reference is never replaced 
     return this;
   }
@@ -547,9 +550,12 @@ public abstract class INodeReference extends INode {
       Snapshot snapshot = null;
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
         snapshot = referred.asFile().getDiffs().getPrior(lastSnapshotId);
-      } else if (referred instanceof INodeDirectoryWithSnapshot) {
-        snapshot = ((INodeDirectoryWithSnapshot) referred).getDiffs().getPrior(
-            lastSnapshotId);
+      } else if (referred.isDirectory()) {
+        DirectoryWithSnapshotFeature sf = referred.asDirectory()
+            .getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          snapshot = sf.getDiffs().getPrior(lastSnapshotId);
+        }
       }
       return snapshot;
     }
@@ -634,10 +640,11 @@ public abstract class INodeReference extends INode {
         Snapshot snapshot = getSelfSnapshot(prior);
         
         INode referred = getReferredINode().asReference().getReferredINode();
-        if (referred.isFile() && referred.asFile().isWithSnapshot()) {
-          // if referred is a file, it must be a file with Snapshot since we did
+        if (referred.isFile()) {
+          // if referred is a file, it must be a file with snapshot since we did
           // recordModification before the rename
           INodeFile file = referred.asFile();
+          Preconditions.checkState(file.isWithSnapshot());
           // make sure we mark the file as deleted
           file.getFileWithSnapshotFeature().deleteCurrentFile();
           try {
@@ -649,14 +656,14 @@ public abstract class INodeReference extends INode {
           } catch (QuotaExceededException e) {
             LOG.error("should not exceed quota while snapshot deletion", e);
           }
-        } else if (referred instanceof INodeDirectoryWithSnapshot) {
+        } else if (referred.isDirectory()) {
           // similarly, if referred is a directory, it must be an
-          // INodeDirectoryWithSnapshot
-          INodeDirectoryWithSnapshot sdir = 
-              (INodeDirectoryWithSnapshot) referred;
+          // INodeDirectory with snapshot
+          INodeDirectory dir = referred.asDirectory();
+          Preconditions.checkState(dir.isWithSnapshot());
           try {
-            INodeDirectoryWithSnapshot.destroyDstSubtree(sdir, snapshot, prior,
-                collectedBlocks, removedINodes);
+            DirectoryWithSnapshotFeature.destroyDstSubtree(dir, snapshot,
+                prior, collectedBlocks, removedINodes);
           } catch (QuotaExceededException e) {
             LOG.error("should not exceed quota while snapshot deletion", e);
           }
@@ -670,9 +677,12 @@ public abstract class INodeReference extends INode {
       Snapshot lastSnapshot = null;
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
         lastSnapshot = referred.asFile().getDiffs().getLastSnapshot();
-      } else if (referred instanceof INodeDirectoryWithSnapshot) {
-        lastSnapshot = ((INodeDirectoryWithSnapshot) referred)
-            .getLastSnapshot();
+      } else if (referred.isDirectory()) {
+        DirectoryWithSnapshotFeature sf = referred.asDirectory()
+            .getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          lastSnapshot = sf.getLastSnapshot();
+        }
       }
       if (lastSnapshot != null && !lastSnapshot.equals(prior)) {
         return lastSnapshot;

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java

@@ -45,11 +45,10 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   }
 
   @Override
-  INode recordModification(Snapshot latest, final INodeMap inodeMap)
-      throws QuotaExceededException {
+  INode recordModification(Snapshot latest) throws QuotaExceededException {
     if (isInLatestSnapshot(latest)) {
       INodeDirectory parent = getParent();
-      parent.saveChild2Snapshot(this, latest, new INodeSymlink(this), inodeMap);
+      parent.saveChild2Snapshot(this, latest, new INodeSymlink(this));
     }
     return this;
   }

+ 47 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java

@@ -95,6 +95,9 @@ public abstract class INodeWithAdditionalFields extends INode
 
   /** For implementing {@link LinkedElement}. */
   private LinkedElement next = null;
+  /** An array {@link Feature}s. */
+  private static final Feature[] EMPTY_FEATURE = new Feature[0];
+  protected Feature[] features = EMPTY_FEATURE;
 
   private INodeWithAdditionalFields(INode parent, long id, byte[] name,
       long permission, long modificationTime, long accessTime) {
@@ -228,13 +231,13 @@ public abstract class INodeWithAdditionalFields extends INode
 
   /** Update modification time if it is larger than the current value. */
   @Override
-  public final INode updateModificationTime(long mtime, Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
+  public final INode updateModificationTime(long mtime, Snapshot latest) 
+      throws QuotaExceededException {
     Preconditions.checkState(isDirectory());
     if (mtime <= modificationTime) {
       return this;
     }
-    return setModificationTime(mtime, latest, inodeMap);
+    return setModificationTime(mtime, latest);
   }
 
   final void cloneModificationTime(INodeWithAdditionalFields that) {
@@ -262,4 +265,45 @@ public abstract class INodeWithAdditionalFields extends INode
   public final void setAccessTime(long accessTime) {
     this.accessTime = accessTime;
   }
+
+  protected void addFeature(Feature f) {
+    int size = features.length;
+    Feature[] arr = new Feature[size + 1];
+    if (size != 0) {
+      System.arraycopy(features, 0, arr, 0, size);
+    }
+    arr[size] = f;
+    features = arr;
+  }
+
+  protected void removeFeature(Feature f) {
+    int size = features.length;
+    Preconditions.checkState(size > 0, "Feature "
+        + f.getClass().getSimpleName() + " not found.");
+
+    if (size == 1) {
+      Preconditions.checkState(features[0] == f, "Feature "
+          + f.getClass().getSimpleName() + " not found.");
+      features = EMPTY_FEATURE;
+      return;
+    }
+
+    Feature[] arr = new Feature[size - 1];
+    int j = 0;
+    boolean overflow = false;
+    for (Feature f1 : features) {
+      if (f1 != f) {
+        if (j == size - 1) {
+          overflow = true;
+          break;
+        } else {
+          arr[j++] = f1;
+        }
+      }
+    }
+
+    Preconditions.checkState(!overflow && j == size - 1, "Feature "
+        + f.getClass().getSimpleName() + " not found.");
+    features = arr;
+  }
 }

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

@@ -26,8 +26,8 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.base.Preconditions;
@@ -132,11 +132,11 @@ public class INodesInPath {
       final boolean isRef = curNode.isReference();
       final boolean isDir = curNode.isDirectory();
       final INodeDirectory dir = isDir? curNode.asDirectory(): null;  
-      if (!isRef && isDir && dir instanceof INodeDirectoryWithSnapshot) {
+      if (!isRef && isDir && dir.isWithSnapshot()) {
         //if the path is a non-snapshot path, update the latest snapshot.
         if (!existing.isSnapshot()) {
-          existing.updateLatestSnapshot(
-              ((INodeDirectoryWithSnapshot)dir).getLastSnapshot());
+          existing.updateLatestSnapshot(dir.getDirectoryWithSnapshotFeature()
+              .getLastSnapshot());
         }
       } else if (isRef && isDir && !lastComp) {
         // If the curNode is a reference node, need to check its dstSnapshot:
@@ -155,10 +155,10 @@ public class INodesInPath {
           if (latest == null ||  // no snapshot in dst tree of rename
               dstSnapshotId >= latest.getId()) { // the above scenario 
             Snapshot lastSnapshot = null;
-            if (curNode.isDirectory()
-                && curNode.asDirectory() instanceof INodeDirectoryWithSnapshot) {
-              lastSnapshot = ((INodeDirectoryWithSnapshot) curNode
-                  .asDirectory()).getLastSnapshot();
+            DirectoryWithSnapshotFeature sf = null;
+            if (curNode.isDirectory() && 
+                (sf = curNode.asDirectory().getDirectoryWithSnapshotFeature()) != null) {
+              lastSnapshot = sf.getLastSnapshot();
             }
             existing.setSnapshot(lastSnapshot);
           }

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

@@ -29,6 +29,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -36,6 +37,7 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -1006,6 +1008,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
       StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
     verifyRequest(nodeReg);
+    metrics.incrBlockReceivedAndDeletedOps();
     if(blockStateChangeLog.isDebugEnabled()) {
       blockStateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: "
           +"from "+nodeReg+" "+receivedAndDeletedBlocks.length
@@ -1238,14 +1241,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override
   public long addCacheDirective(
-      CacheDirectiveInfo path) throws IOException {
-    return namesystem.addCacheDirective(path);
+      CacheDirectiveInfo path, EnumSet<CacheFlag> flags) throws IOException {
+    return namesystem.addCacheDirective(path, flags);
   }
 
   @Override
   public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
-    namesystem.modifyCacheDirective(directive);
+      CacheDirectiveInfo directive, EnumSet<CacheFlag> flags) throws IOException {
+    namesystem.modifyCacheDirective(directive, flags);
   }
 
   @Override

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

@@ -71,6 +71,8 @@ public class NameNodeMetrics {
   MutableCounterLong listSnapshottableDirOps;
   @Metric("Number of snapshotDiffReport operations")
   MutableCounterLong snapshotDiffReportOps;
+  @Metric("Number of blockReceivedAndDeleted calls")
+  MutableCounterLong blockReceivedAndDeletedOps;
 
   @Metric("Journal transactions") MutableRate transactions;
   @Metric("Journal syncs") MutableRate syncs;
@@ -87,6 +89,13 @@ public class NameNodeMetrics {
   @Metric("Time loading FS Image at startup in msec")
   MutableGaugeInt fsImageLoadTime;
 
+  @Metric("GetImageServlet getEdit")
+  MutableRate getEdit;
+  @Metric("GetImageServlet getImage")
+  MutableRate getImage;
+  @Metric("GetImageServlet putImage")
+  MutableRate putImage;
+
   NameNodeMetrics(String processName, String sessionId, int[] intervals) {
     registry.tag(ProcessName, processName).tag(SessionId, sessionId);
     
@@ -209,6 +218,10 @@ public class NameNodeMetrics {
     snapshotDiffReportOps.incr();
   }
   
+  public void incrBlockReceivedAndDeletedOps() {
+    blockReceivedAndDeletedOps.incr();
+  }
+
   public void addTransaction(long latency) {
     transactions.add(latency);
   }
@@ -245,4 +258,16 @@ public class NameNodeMetrics {
   public void setSafeModeTime(long elapsed) {
     safeModeTime.set((int) elapsed);
   }
+
+  public void addGetEdit(long latency) {
+    getEdit.add(latency);
+  }
+
+  public void addGetImage(long latency) {
+    getImage.add(latency);
+  }
+
+  public void addPutImage(long latency) {
+    putImage.add(latency);
+  }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java

@@ -98,7 +98,7 @@ abstract class AbstractINodeDiff<N extends INode,
   }
 
   /** Save the INode state to the snapshot if it is not done already. */
-  void saveSnapshotCopy(A snapshotCopy, N currentINode) {
+  void saveSnapshotCopy(A snapshotCopy) {
     Preconditions.checkState(snapshotINode == null, "Expected snapshotINode to be null");
     snapshotINode = snapshotCopy;
   }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java

@@ -25,8 +25,8 @@ import java.util.List;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
@@ -271,7 +271,7 @@ abstract class AbstractINodeDiffList<N extends INode,
    *         Note that the current inode is returned if there is no change
    *         between the given snapshot and the current state. 
    */
-  A getSnapshotINode(final Snapshot snapshot, final A currentINode) {
+  public A getSnapshotINode(final Snapshot snapshot, final A currentINode) {
     final D diff = getDiff(snapshot);
     final A inode = diff == null? null: diff.getSnapshotINode();
     return inode == null? currentINode: inode;
@@ -306,7 +306,7 @@ abstract class AbstractINodeDiffList<N extends INode,
         if (snapshotCopy == null) {
           snapshotCopy = createSnapshotCopy(currentINode);
         }
-        diff.saveSnapshotCopy(snapshotCopy, currentINode);
+        diff.saveSnapshotCopy(snapshotCopy);
       }
     }
   }

+ 321 - 462
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java

@@ -28,6 +28,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
@@ -35,10 +36,10 @@ import org.apache.hadoop.hdfs.server.namenode.Content;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -51,18 +52,17 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import com.google.common.base.Preconditions;
 
 /**
- * The directory with snapshots. It maintains a list of snapshot diffs for
- * storing snapshot data. When there are modifications to the directory, the old
- * data is stored in the latest snapshot, if there is any.
+ * Feature for directory with snapshot-related information.
  */
-public class INodeDirectoryWithSnapshot extends INodeDirectory {
+@InterfaceAudience.Private
+public class DirectoryWithSnapshotFeature implements INode.Feature {
   /**
    * The difference between the current state and a previous snapshot
    * of the children list of an INodeDirectory.
    */
   static class ChildrenDiff extends Diff<byte[], INode> {
     ChildrenDiff() {}
-    
+
     private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
       super(created, deleted);
     }
@@ -73,7 +73,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
      */
     private final boolean replace(final ListType type,
         final INode oldChild, final INode newChild) {
-      final List<INode> list = getList(type); 
+      final List<INode> list = getList(type);
       final int i = search(list, oldChild.getLocalNameBytes());
       if (i < 0 || list.get(i).getId() != oldChild.getId()) {
         return false;
@@ -93,10 +93,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       }
       return false;
     }
-    
+
     /** clear the created list */
-    private Quota.Counts destroyCreatedList(
-        final INodeDirectoryWithSnapshot currentINode,
+    private Quota.Counts destroyCreatedList(final INodeDirectory currentINode,
         final BlocksMapUpdateInfo collectedBlocks,
         final List<INode> removedINodes) {
       Quota.Counts counts = Quota.Counts.newInstance();
@@ -110,7 +109,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       createdList.clear();
       return counts;
     }
-    
+
     /** clear the deleted list */
     private Quota.Counts destroyDeletedList(
         final BlocksMapUpdateInfo collectedBlocks,
@@ -124,19 +123,19 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       deletedList.clear();
       return counts;
     }
-    
+
     /** Serialize {@link #created} */
     private void writeCreated(DataOutput out) throws IOException {
       final List<INode> created = getList(ListType.CREATED);
       out.writeInt(created.size());
       for (INode node : created) {
-        // For INode in created list, we only need to record its local name 
+        // For INode in created list, we only need to record its local name
         byte[] name = node.getLocalNameBytes();
         out.writeShort(name.length);
         out.write(name);
       }
     }
-    
+
     /** Serialize {@link #deleted} */
     private void writeDeleted(DataOutput out,
         ReferenceMap referenceMap) throws IOException {
@@ -146,12 +145,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         FSImageSerialization.saveINode2Image(node, out, true, referenceMap);
       }
     }
-    
+
     /** Serialize to out */
     private void write(DataOutput out, ReferenceMap referenceMap
         ) throws IOException {
       writeCreated(out);
-      writeDeleted(out, referenceMap);    
+      writeDeleted(out, referenceMap);
     }
 
     /** Get the list of INodeDirectory contained in the deleted list */
@@ -162,17 +161,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
       }
     }
-    
+
     /**
      * Interpret the diff and generate a list of {@link DiffReportEntry}.
      * @param parentPath The relative path of the parent.
-     * @param parent The directory that the diff belongs to.
-     * @param fromEarlier True indicates {@code diff=later-earlier}, 
+     * @param fromEarlier True indicates {@code diff=later-earlier},
      *                    False indicates {@code diff=earlier-later}
      * @return A list of {@link DiffReportEntry} as the diff report.
      */
     public List<DiffReportEntry> generateReport(byte[][] parentPath,
-        INodeDirectoryWithSnapshot parent, boolean fromEarlier) {
+        boolean fromEarlier) {
       List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
       List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
       int c = 0, d = 0;
@@ -217,7 +215,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       return dList;
     }
   }
-  
+
   /**
    * The difference of an {@link INodeDirectory} between two snapshots.
    */
@@ -243,16 +241,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       this.childrenSize = childrenSize;
       this.diff = new ChildrenDiff(createdList, deletedList);
     }
-    
+
     ChildrenDiff getChildrenDiff() {
       return diff;
     }
-    
+
     /** Is the inode the root of the snapshot? */
     boolean isSnapshotRoot() {
       return snapshotINode == snapshot.getRoot();
     }
-    
+
     @Override
     Quota.Counts combinePosteriorAndCollectBlocks(
         final INodeDirectory currentDir, final DirectoryDiff posterior,
@@ -277,14 +275,15 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
      *         Since the snapshot is read-only, the logical view of the list is
      *         never changed although the internal data structure may mutate.
      */
-    ReadOnlyList<INode> getChildrenList(final INodeDirectory currentDir) {
+    private ReadOnlyList<INode> getChildrenList(final INodeDirectory currentDir) {
       return new ReadOnlyList<INode>() {
         private List<INode> children = null;
 
         private List<INode> initChildren() {
           if (children == null) {
             final ChildrenDiff combined = new ChildrenDiff();
-            for(DirectoryDiff d = DirectoryDiff.this; d != null; d = d.getPosterior()) {
+            for (DirectoryDiff d = DirectoryDiff.this; d != null; 
+                d = d.getPosterior()) {
               combined.combinePosterior(d.diff, null);
             }
             children = combined.apply2Current(ReadOnlyList.Util.asList(
@@ -297,17 +296,17 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         public Iterator<INode> iterator() {
           return initChildren().iterator();
         }
-    
+
         @Override
         public boolean isEmpty() {
           return childrenSize == 0;
         }
-    
+
         @Override
         public int size() {
           return childrenSize;
         }
-    
+
         @Override
         public INode get(int i) {
           return initChildren().get(i);
@@ -322,9 +321,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         final Container<INode> returned = d.diff.accessPrevious(name);
         if (returned != null) {
           // the diff is able to determine the inode
-          return returned.getElement(); 
+          return returned.getElement();
         } else if (!checkPosterior) {
-          // Since checkPosterior is false, return null, i.e. not found.   
+          // Since checkPosterior is false, return null, i.e. not found.
           return null;
         } else if (d.getPosterior() == null) {
           // no more posterior diff, get from current inode.
@@ -332,12 +331,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
       }
     }
-    
+
     @Override
     public String toString() {
       return super.toString() + " childrenSize=" + childrenSize + ", " + diff;
     }
-    
+
     @Override
     void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
       writeSnapshot(out);
@@ -386,7 +385,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
     }
 
     /** Replace the given child in the created/deleted list, if there is any. */
-    private boolean replaceChild(final ListType type, final INode oldChild,
+    public boolean replaceChild(final ListType type, final INode oldChild,
         final INode newChild) {
       final List<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
@@ -397,9 +396,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       }
       return false;
     }
-    
+
     /** Remove the given child in the created/deleted list, if there is any. */
-    private boolean removeChild(final ListType type, final INode child) {
+    public boolean removeChild(final ListType type, final INode child) {
       final List<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
         final ChildrenDiff diff = diffList.get(i).diff;
@@ -410,84 +409,134 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       return false;
     }
   }
-
-  /**
-   * Compute the difference between Snapshots.
-   * 
-   * @param fromSnapshot Start point of the diff computation. Null indicates
-   *          current tree.
-   * @param toSnapshot End point of the diff computation. Null indicates current
-   *          tree.
-   * @param diff Used to capture the changes happening to the children. Note
-   *          that the diff still represents (later_snapshot - earlier_snapshot)
-   *          although toSnapshot can be before fromSnapshot.
-   * @return Whether changes happened between the startSnapshot and endSnaphsot.
-   */
-  boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
-      Snapshot toSnapshot, ChildrenDiff diff) {
-    Snapshot earlier = fromSnapshot;
-    Snapshot later = toSnapshot;
-    if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
-      earlier = toSnapshot;
-      later = fromSnapshot;
+  
+  private static Map<INode, INode> cloneDiffList(List<INode> diffList) {
+    if (diffList == null || diffList.size() == 0) {
+      return null;
     }
-    
-    boolean modified = diffs.changedBetweenSnapshots(earlier,
-        later);
-    if (!modified) {
-      return false;
+    Map<INode, INode> map = new HashMap<INode, INode>(diffList.size());
+    for (INode node : diffList) {
+      map.put(node, node);
     }
-    
-    final List<DirectoryDiff> difflist = diffs.asList();
-    final int size = difflist.size();
-    int earlierDiffIndex = Collections.binarySearch(difflist, earlier.getId());
-    int laterDiffIndex = later == null ? size : Collections
-        .binarySearch(difflist, later.getId());
-    earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
-        : earlierDiffIndex;
-    laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
-        : laterDiffIndex;
-    
-    boolean dirMetadataChanged = false;
-    INodeDirectoryAttributes dirCopy = null;
-    for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
-      DirectoryDiff sdiff = difflist.get(i);
-      diff.combinePosterior(sdiff.diff, null);
-      if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
-        if (dirCopy == null) {
-          dirCopy = sdiff.snapshotINode;
-        } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
-          dirMetadataChanged = true;
+    return map;
+  }
+  
+  /**
+   * Destroy a subtree under a DstReference node.
+   */
+  public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
+      final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) throws QuotaExceededException {
+    Preconditions.checkArgument(prior != null);
+    if (inode.isReference()) {
+      if (inode instanceof INodeReference.WithName && snapshot != null) {
+        // this inode has been renamed before the deletion of the DstReference
+        // subtree
+        inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes,
+            true);
+      } else { 
+        // for DstReference node, continue this process to its subtree
+        destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
+            prior, collectedBlocks, removedINodes);
+      }
+    } else if (inode.isFile()) {
+      inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes, true);
+    } else if (inode.isDirectory()) {
+      Map<INode, INode> excludedNodes = null;
+      INodeDirectory dir = inode.asDirectory();
+      DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
+      if (sf != null) {
+        DirectoryDiffList diffList = sf.getDiffs();
+        DirectoryDiff priorDiff = diffList.getDiff(prior);
+        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
+          List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
+          excludedNodes = cloneDiffList(dList);
+        }
+        
+        if (snapshot != null) {
+          diffList.deleteSnapshotDiff(snapshot, prior, dir, collectedBlocks,
+              removedINodes, true);
+        }
+        priorDiff = diffList.getDiff(prior);
+        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
+          priorDiff.diff.destroyCreatedList(dir, collectedBlocks,
+              removedINodes);
         }
       }
+      for (INode child : inode.asDirectory().getChildrenList(prior)) {
+        if (excludedNodes != null && excludedNodes.containsKey(child)) {
+          continue;
+        }
+        destroyDstSubtree(child, snapshot, prior, collectedBlocks,
+            removedINodes);
+      }
     }
-
-    if (!diff.isEmpty() || dirMetadataChanged) {
-      return true;
-    } else if (dirCopy != null) {
-      for (int i = laterDiffIndex; i < size; i++) {
-        if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
-          return true;
+  }
+  
+  /**
+   * Clean an inode while we move it from the deleted list of post to the
+   * deleted list of prior.
+   * @param inode The inode to clean.
+   * @param post The post snapshot.
+   * @param prior The prior snapshot.
+   * @param collectedBlocks Used to collect blocks for later deletion.
+   * @return Quota usage update.
+   */
+  private static Quota.Counts cleanDeletedINode(INode inode,
+      final Snapshot post, final Snapshot prior,
+      final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes, final boolean countDiffChange) 
+      throws QuotaExceededException {
+    Quota.Counts counts = Quota.Counts.newInstance();
+    Deque<INode> queue = new ArrayDeque<INode>();
+    queue.addLast(inode);
+    while (!queue.isEmpty()) {
+      INode topNode = queue.pollFirst();
+      if (topNode instanceof INodeReference.WithName) {
+        INodeReference.WithName wn = (INodeReference.WithName) topNode;
+        if (wn.getLastSnapshotId() >= post.getId()) {
+          wn.cleanSubtree(post, prior, collectedBlocks, removedINodes,
+              countDiffChange);
+        }
+        // For DstReference node, since the node is not in the created list of
+        // prior, we should treat it as regular file/dir
+      } else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
+        INodeFile file = topNode.asFile();
+        counts.add(file.getDiffs().deleteSnapshotDiff(post, prior, file,
+            collectedBlocks, removedINodes, countDiffChange));
+      } else if (topNode.isDirectory()) {
+        INodeDirectory dir = topNode.asDirectory();
+        ChildrenDiff priorChildrenDiff = null;
+        DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          // delete files/dirs created after prior. Note that these
+          // files/dirs, along with inode, were deleted right after post.
+          DirectoryDiff priorDiff = sf.getDiffs().getDiff(prior);
+          if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
+            priorChildrenDiff = priorDiff.getChildrenDiff();
+            counts.add(priorChildrenDiff.destroyCreatedList(dir,
+                collectedBlocks, removedINodes));
+          }
+        }
+        
+        for (INode child : dir.getChildrenList(prior)) {
+          if (priorChildrenDiff != null
+              && priorChildrenDiff.search(ListType.DELETED,
+                  child.getLocalNameBytes()) != null) {
+            continue;
+          }
+          queue.addLast(child);
         }
       }
-      return !dirCopy.metadataEquals(this);
-    } else {
-      return false;
     }
+    return counts;
   }
 
   /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
   private final DirectoryDiffList diffs;
 
-  public INodeDirectoryWithSnapshot(INodeDirectory that) {
-    this(that, true, that instanceof INodeDirectoryWithSnapshot?
-        ((INodeDirectoryWithSnapshot)that).getDiffs(): null);
-  }
-
-  INodeDirectoryWithSnapshot(INodeDirectory that, boolean adopt,
-      DirectoryDiffList diffs) {
-    super(that, adopt, true);
-    this.diffs = diffs != null? diffs: new DirectoryDiffList();
+  public DirectoryWithSnapshotFeature(DirectoryDiffList diffs) {
+    this.diffs = diffs != null ? diffs : new DirectoryDiffList();
   }
 
   /** @return the last snapshot. */
@@ -499,204 +548,203 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
   public DirectoryDiffList getDiffs() {
     return diffs;
   }
-
-  @Override
-  public INodeDirectoryAttributes getSnapshotINode(Snapshot snapshot) {
-    return diffs.getSnapshotINode(snapshot, this);
-  }
-
-  @Override
-  public INodeDirectoryWithSnapshot recordModification(final Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
-      return saveSelf2Snapshot(latest, null);
+  
+  /**
+   * Get all the directories that are stored in some snapshot but not in the
+   * current children list. These directories are equivalent to the directories
+   * stored in the deletes lists.
+   */
+  public void getSnapshotDirectory(List<INodeDirectory> snapshotDir) {
+    for (DirectoryDiff sdiff : diffs) {
+      sdiff.getChildrenDiff().getDirsInDeleted(snapshotDir);
     }
-    return this;
-  }
-
-  /** Save the snapshot copy to the latest snapshot. */
-  public INodeDirectoryWithSnapshot saveSelf2Snapshot(
-      final Snapshot latest, final INodeDirectory snapshotCopy)
-          throws QuotaExceededException {
-    diffs.saveSelf2Snapshot(latest, this, snapshotCopy);
-    return this;
   }
 
-  @Override
-  public INode saveChild2Snapshot(final INode child, final Snapshot latest,
-      final INode snapshotCopy, final INodeMap inodeMap)
-      throws QuotaExceededException {
-    Preconditions.checkArgument(!child.isDirectory(),
-        "child is a directory, child=%s", child);
-    if (latest == null) {
-      return child;
-    }
-
-    final DirectoryDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, this);
-    if (diff.getChild(child.getLocalNameBytes(), false, this) != null) {
-      // it was already saved in the latest snapshot earlier.  
-      return child;
-    }
-
-    diff.diff.modify(snapshotCopy, child);
-    return child;
-  }
+  /**
+   * Add an inode into parent's children list. The caller of this method needs
+   * to make sure that parent is in the given snapshot "latest".
+   */
+  public boolean addChild(INodeDirectory parent, INode inode,
+      boolean setModTime, Snapshot latest) throws QuotaExceededException {
+    ChildrenDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, parent).diff;
+    int undoInfo = diff.create(inode);
 
-  @Override
-  public boolean addChild(INode inode, boolean setModTime, Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    ChildrenDiff diff = null;
-    Integer undoInfo = null;
-    if (isInLatestSnapshot(latest)) {
-      diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
-      undoInfo = diff.create(inode);
-    }
-    final boolean added = super.addChild(inode, setModTime, null, inodeMap);
-    if (!added && undoInfo != null) {
+    final boolean added = parent.addChild(inode, setModTime, null);
+    if (!added) {
       diff.undoCreate(inode, undoInfo);
     }
-    return added; 
+    return added;
   }
 
-  @Override
-  public boolean removeChild(INode child, Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    ChildrenDiff diff = null;
-    UndoInfo<INode> undoInfo = null;
+  /**
+   * Remove an inode from parent's children list. The caller of this method
+   * needs to make sure that parent is in the given snapshot "latest".
+   */
+  public boolean removeChild(INodeDirectory parent, INode child,
+      Snapshot latest) throws QuotaExceededException {
     // For a directory that is not a renamed node, if isInLatestSnapshot returns
     // false, the directory is not in the latest snapshot, thus we do not need
     // to record the removed child in any snapshot.
     // For a directory that was moved/renamed, note that if the directory is in
-    // any of the previous snapshots, we will create a reference node for the 
+    // any of the previous snapshots, we will create a reference node for the
     // directory while rename, and isInLatestSnapshot will return true in that
     // scenario (if all previous snapshots have been deleted, isInLatestSnapshot
-    // still returns false). Thus if isInLatestSnapshot returns false, the 
-    // directory node cannot be in any snapshot (not in current tree, nor in 
-    // previous src tree). Thus we do not need to record the removed child in 
+    // still returns false). Thus if isInLatestSnapshot returns false, the
+    // directory node cannot be in any snapshot (not in current tree, nor in
+    // previous src tree). Thus we do not need to record the removed child in
     // any snapshot.
-    if (isInLatestSnapshot(latest)) {
-      diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
-      undoInfo = diff.delete(child);
-    }
-    final boolean removed = removeChild(child);
-    if (undoInfo != null) {
-      if (!removed) {
-        //remove failed, undo
-        diff.undoDelete(child, undoInfo);
-      }
+    ChildrenDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, parent).diff;
+    UndoInfo<INode> undoInfo = diff.delete(child);
+
+    final boolean removed = parent.removeChild(child);
+    if (!removed && undoInfo != null) {
+      // remove failed, undo
+      diff.undoDelete(child, undoInfo);
     }
     return removed;
   }
   
-  @Override
-  public void replaceChild(final INode oldChild, final INode newChild,
-      final INodeMap inodeMap) {
-    super.replaceChild(oldChild, newChild, inodeMap);
-    if (oldChild.getParentReference() != null && !newChild.isReference()) {
-      // oldChild is referred by a Reference node. Thus we are replacing the 
-      // referred inode, e.g., 
-      // INodeFileWithSnapshot -> INodeFileUnderConstructionWithSnapshot
-      // in this case, we do not need to update the diff list
-      return;
-    } else {
-      diffs.replaceChild(ListType.CREATED, oldChild, newChild);
-    }
-  }
-  
   /**
-   * This method is usually called by the undo section of rename.
-   * 
-   * Before calling this function, in the rename operation, we replace the
-   * original src node (of the rename operation) with a reference node (WithName
-   * instance) in both the children list and a created list, delete the
-   * reference node from the children list, and add it to the corresponding
-   * deleted list.
-   * 
-   * To undo the above operations, we have the following steps in particular:
-   * 
-   * <pre>
-   * 1) remove the WithName node from the deleted list (if it exists) 
-   * 2) replace the WithName node in the created list with srcChild 
-   * 3) add srcChild back as a child of srcParent. Note that we already add 
-   * the node into the created list of a snapshot diff in step 2, we do not need
-   * to add srcChild to the created list of the latest snapshot.
-   * </pre>
-   * 
-   * We do not need to update quota usage because the old child is in the 
-   * deleted list before. 
-   * 
-   * @param oldChild
-   *          The reference node to be removed/replaced
-   * @param newChild
-   *          The node to be added back
-   * @param latestSnapshot
-   *          The latest snapshot. Note this may not be the last snapshot in the
-   *          {@link #diffs}, since the src tree of the current rename operation
-   *          may be the dst tree of a previous rename.
-   * @throws QuotaExceededException should not throw this exception
+   * @return If there is no corresponding directory diff for the given
+   *         snapshot, this means that the current children list should be
+   *         returned for the snapshot. Otherwise we calculate the children list
+   *         for the snapshot and return it. 
    */
-  public void undoRename4ScrParent(final INodeReference oldChild,
-      final INode newChild, Snapshot latestSnapshot)
-      throws QuotaExceededException {
-    diffs.removeChild(ListType.DELETED, oldChild);
-    diffs.replaceChild(ListType.CREATED, oldChild, newChild);
-    // pass null for inodeMap since the parent node will not get replaced when
-    // undoing rename
-    addChild(newChild, true, null, null);
+  public ReadOnlyList<INode> getChildrenList(INodeDirectory currentINode,
+      final Snapshot snapshot) {
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
+    return diff != null ? diff.getChildrenList(currentINode) : currentINode
+        .getChildrenList(null);
   }
   
-  /**
-   * Undo the rename operation for the dst tree, i.e., if the rename operation
-   * (with OVERWRITE option) removes a file/dir from the dst tree, add it back
-   * and delete possible record in the deleted list.  
-   */
-  public void undoRename4DstParent(final INode deletedChild,
-      Snapshot latestSnapshot) throws QuotaExceededException {
-    boolean removeDeletedChild = diffs.removeChild(ListType.DELETED,
-        deletedChild);
-    // pass null for inodeMap since the parent node will not get replaced when
-    // undoing rename
-    final boolean added = addChild(deletedChild, true, removeDeletedChild ? null
-        : latestSnapshot, null);
-    // update quota usage if adding is successfully and the old child has not
-    // been stored in deleted list before
-    if (added && !removeDeletedChild) {
-      final Quota.Counts counts = deletedChild.computeQuotaUsage();
-      addSpaceConsumed(counts.get(Quota.NAMESPACE),
-          counts.get(Quota.DISKSPACE), false);
-    }
-  }
-
-  @Override
-  public ReadOnlyList<INode> getChildrenList(Snapshot snapshot) {
+  public INode getChild(INodeDirectory currentINode, byte[] name,
+      Snapshot snapshot) {
     final DirectoryDiff diff = diffs.getDiff(snapshot);
-    return diff != null? diff.getChildrenList(this): super.getChildrenList(null);
+    return diff != null ? diff.getChild(name, true, currentINode)
+        : currentINode.getChild(name, null);
   }
+  
+  /** Used to record the modification of a symlink node */
+  public INode saveChild2Snapshot(INodeDirectory currentINode,
+      final INode child, final Snapshot latest, final INode snapshotCopy)
+      throws QuotaExceededException {
+    Preconditions.checkArgument(!child.isDirectory(),
+        "child is a directory, child=%s", child);
+    Preconditions.checkArgument(latest != null);
+    
+    final DirectoryDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest,
+        currentINode);
+    if (diff.getChild(child.getLocalNameBytes(), false, currentINode) != null) {
+      // it was already saved in the latest snapshot earlier.  
+      return child;
+    }
 
-  @Override
-  public INode getChild(byte[] name, Snapshot snapshot) {
-    final DirectoryDiff diff = diffs.getDiff(snapshot);
-    return diff != null? diff.getChild(name, true, this): super.getChild(name, null);
+    diff.diff.modify(snapshotCopy, child);
+    return child;
   }
-
-  @Override
-  public String toDetailString() {
-    return super.toDetailString() + ", " + diffs;
+  
+  public void clear(INodeDirectory currentINode,
+      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+    // destroy its diff list
+    for (DirectoryDiff diff : diffs) {
+      diff.destroyDiffAndCollectBlocks(currentINode, collectedBlocks,
+          removedINodes);
+    }
+    diffs.clear();
+  }
+  
+  public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
+    for(DirectoryDiff d : diffs) {
+      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
+        deleted.computeQuotaUsage(counts, false, Snapshot.INVALID_ID);
+      }
+    }
+    counts.add(Quota.NAMESPACE, diffs.asList().size());
+    return counts;
+  }
+  
+  public void computeContentSummary4Snapshot(final Content.Counts counts) {
+    // Create a new blank summary context for blocking processing of subtree.
+    ContentSummaryComputationContext summary = 
+        new ContentSummaryComputationContext();
+    for(DirectoryDiff d : diffs) {
+      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
+        deleted.computeContentSummary(summary);
+      }
+    }
+    // Add the counts from deleted trees.
+    counts.add(summary.getCounts());
+    // Add the deleted directory count.
+    counts.add(Content.DIRECTORY, diffs.asList().size());
   }
   
   /**
-   * Get all the directories that are stored in some snapshot but not in the
-   * current children list. These directories are equivalent to the directories
-   * stored in the deletes lists.
+   * Compute the difference between Snapshots.
+   *
+   * @param fromSnapshot Start point of the diff computation. Null indicates
+   *          current tree.
+   * @param toSnapshot End point of the diff computation. Null indicates current
+   *          tree.
+   * @param diff Used to capture the changes happening to the children. Note
+   *          that the diff still represents (later_snapshot - earlier_snapshot)
+   *          although toSnapshot can be before fromSnapshot.
+   * @param currentINode The {@link INodeDirectory} this feature belongs to.
+   * @return Whether changes happened between the startSnapshot and endSnaphsot.
    */
-  public void getSnapshotDirectory(List<INodeDirectory> snapshotDir) {
-    for (DirectoryDiff sdiff : diffs) {
-      sdiff.getChildrenDiff().getDirsInDeleted(snapshotDir);
+  boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
+      Snapshot toSnapshot, ChildrenDiff diff, INodeDirectory currentINode) {
+    Snapshot earlier = fromSnapshot;
+    Snapshot later = toSnapshot;
+    if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
+      earlier = toSnapshot;
+      later = fromSnapshot;
+    }
+
+    boolean modified = diffs.changedBetweenSnapshots(earlier, later);
+    if (!modified) {
+      return false;
+    }
+
+    final List<DirectoryDiff> difflist = diffs.asList();
+    final int size = difflist.size();
+    int earlierDiffIndex = Collections.binarySearch(difflist, earlier.getId());
+    int laterDiffIndex = later == null ? size : Collections
+        .binarySearch(difflist, later.getId());
+    earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
+        : earlierDiffIndex;
+    laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
+        : laterDiffIndex;
+
+    boolean dirMetadataChanged = false;
+    INodeDirectoryAttributes dirCopy = null;
+    for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
+      DirectoryDiff sdiff = difflist.get(i);
+      diff.combinePosterior(sdiff.diff, null);
+      if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
+        if (dirCopy == null) {
+          dirCopy = sdiff.snapshotINode;
+        } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
+          dirMetadataChanged = true;
+        }
+      }
+    }
+
+    if (!diff.isEmpty() || dirMetadataChanged) {
+      return true;
+    } else if (dirCopy != null) {
+      for (int i = laterDiffIndex; i < size; i++) {
+        if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
+          return true;
+        }
+      }
+      return !dirCopy.metadataEquals(currentINode);
+    } else {
+      return false;
     }
   }
 
-  @Override
-  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
+  public Quota.Counts cleanDirectory(final INodeDirectory currentINode,
+      final Snapshot snapshot, Snapshot prior,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes, final boolean countDiffChange)
       throws QuotaExceededException {
@@ -704,12 +752,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
     Map<INode, INode> priorCreated = null;
     Map<INode, INode> priorDeleted = null;
     if (snapshot == null) { // delete the current directory
-      recordModification(prior, null);
+      currentINode.recordModification(prior);
       // delete everything in created list
       DirectoryDiff lastDiff = diffs.getLast();
       if (lastDiff != null) {
-        counts.add(lastDiff.diff.destroyCreatedList(this, collectedBlocks,
-            removedINodes));
+        counts.add(lastDiff.diff.destroyCreatedList(currentINode,
+            collectedBlocks, removedINodes));
       }
     } else {
       // update prior
@@ -726,7 +774,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
       }
       
-      counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior, this, 
+      counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior, currentINode, 
           collectedBlocks, removedINodes, countDiffChange));
       
       // check priorDiff again since it may be created during the diff deletion
@@ -767,202 +815,13 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
       }
     }
-    counts.add(cleanSubtreeRecursively(snapshot, prior, collectedBlocks,
-        removedINodes, priorDeleted, countDiffChange));
+    counts.add(currentINode.cleanSubtreeRecursively(snapshot, prior,
+        collectedBlocks, removedINodes, priorDeleted, countDiffChange));
     
-    if (isQuotaSet()) {
-      getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
+    if (currentINode.isQuotaSet()) {
+      currentINode.getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
     }
     return counts;
   }
-  
-  /**
-   * Clean an inode while we move it from the deleted list of post to the
-   * deleted list of prior.
-   * @param inode The inode to clean.
-   * @param post The post snapshot.
-   * @param prior The prior snapshot.
-   * @param collectedBlocks Used to collect blocks for later deletion.
-   * @return Quota usage update.
-   */
-  private static Quota.Counts cleanDeletedINode(INode inode,
-      final Snapshot post, final Snapshot prior,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final boolean countDiffChange) 
-      throws QuotaExceededException {
-    Quota.Counts counts = Quota.Counts.newInstance();
-    Deque<INode> queue = new ArrayDeque<INode>();
-    queue.addLast(inode);
-    while (!queue.isEmpty()) {
-      INode topNode = queue.pollFirst();
-      if (topNode instanceof INodeReference.WithName) {
-        INodeReference.WithName wn = (INodeReference.WithName) topNode;
-        if (wn.getLastSnapshotId() >= post.getId()) {
-          wn.cleanSubtree(post, prior, collectedBlocks, removedINodes,
-              countDiffChange);
-        }
-        // For DstReference node, since the node is not in the created list of
-        // prior, we should treat it as regular file/dir
-      } else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
-        INodeFile file = topNode.asFile();
-        counts.add(file.getDiffs().deleteSnapshotDiff(post, prior, file,
-            collectedBlocks, removedINodes, countDiffChange));
-      } else if (topNode.isDirectory()) {
-        INodeDirectory dir = topNode.asDirectory();
-        ChildrenDiff priorChildrenDiff = null;
-        if (dir instanceof INodeDirectoryWithSnapshot) {
-          // delete files/dirs created after prior. Note that these
-          // files/dirs, along with inode, were deleted right after post.
-          INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
-          DirectoryDiff priorDiff = sdir.getDiffs().getDiff(prior);
-          if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
-            priorChildrenDiff = priorDiff.getChildrenDiff();
-            counts.add(priorChildrenDiff.destroyCreatedList(sdir,
-                collectedBlocks, removedINodes));
-          }
-        }
-        
-        for (INode child : dir.getChildrenList(prior)) {
-          if (priorChildrenDiff != null
-              && priorChildrenDiff.search(ListType.DELETED,
-                  child.getLocalNameBytes()) != null) {
-            continue;
-          }
-          queue.addLast(child);
-        }
-      }
-    }
-    return counts;
-  }
-
-  @Override
-  public void destroyAndCollectBlocks(
-      final BlocksMapUpdateInfo collectedBlocks, 
-      final List<INode> removedINodes) {
-    // destroy its diff list
-    for (DirectoryDiff diff : diffs) {
-      diff.destroyDiffAndCollectBlocks(this, collectedBlocks, removedINodes);
-    }
-    diffs.clear();
-    super.destroyAndCollectBlocks(collectedBlocks, removedINodes);
-  }
-
-  @Override
-  public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
-      boolean useCache, int lastSnapshotId) {
-    if ((useCache && isQuotaSet()) || lastSnapshotId == Snapshot.INVALID_ID) {
-      return super.computeQuotaUsage(counts, useCache, lastSnapshotId);
-    }
-    
-    Snapshot lastSnapshot = diffs.getSnapshotById(lastSnapshotId);
-    
-    ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshot);
-    for (INode child : childrenList) {
-      child.computeQuotaUsage(counts, useCache, lastSnapshotId);
-    }
-    
-    counts.add(Quota.NAMESPACE, 1);
-    return counts;
-  }
-  
-  @Override
-  public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
-    super.computeQuotaUsage4CurrentDirectory(counts);
-    for(DirectoryDiff d : diffs) {
-      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        deleted.computeQuotaUsage(counts, false, Snapshot.INVALID_ID);
-      }
-    }
-    counts.add(Quota.NAMESPACE, diffs.asList().size());
-    return counts;
-  }
-
-  @Override
-  public ContentSummaryComputationContext computeContentSummary(
-      final ContentSummaryComputationContext summary) {
-    // Snapshot summary calc won't be relinquishing locks in the middle.
-    // Do this first and handover to parent.
-    computeContentSummary4Snapshot(summary.getCounts());
-    super.computeContentSummary(summary);
-    return summary;
-  }
-
-  private void computeContentSummary4Snapshot(final Content.Counts counts) {
-    // Create a new blank summary context for blocking processing of subtree.
-    ContentSummaryComputationContext summary = 
-        new ContentSummaryComputationContext();
-    for(DirectoryDiff d : diffs) {
-      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        deleted.computeContentSummary(summary);
-      }
-    }
-    // Add the counts from deleted trees.
-    counts.add(summary.getCounts());
-    // Add the deleted directory count.
-    counts.add(Content.DIRECTORY, diffs.asList().size());
-  }
-  
-  private static Map<INode, INode> cloneDiffList(List<INode> diffList) {
-    if (diffList == null || diffList.size() == 0) {
-      return null;
-    }
-    Map<INode, INode> map = new HashMap<INode, INode>(diffList.size());
-    for (INode node : diffList) {
-      map.put(node, node);
-    }
-    return map;
-  }
-  
-  /**
-   * Destroy a subtree under a DstReference node.
-   */
-  public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
-      final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes) throws QuotaExceededException {
-    Preconditions.checkArgument(prior != null);
-    if (inode.isReference()) {
-      if (inode instanceof INodeReference.WithName && snapshot != null) {
-        // this inode has been renamed before the deletion of the DstReference
-        // subtree
-        inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes,
-            true);
-      } else { 
-        // for DstReference node, continue this process to its subtree
-        destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
-            prior, collectedBlocks, removedINodes);
-      }
-    } else if (inode.isFile()) {
-      inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes, true);
-    } else if (inode.isDirectory()) {
-      Map<INode, INode> excludedNodes = null;
-      if (inode instanceof INodeDirectoryWithSnapshot) {
-        INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) inode;
-        
-        DirectoryDiffList diffList = sdir.getDiffs();
-        DirectoryDiff priorDiff = diffList.getDiff(prior);
-        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
-          List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
-          excludedNodes = cloneDiffList(dList);
-        }
-        
-        if (snapshot != null) {
-          diffList.deleteSnapshotDiff(snapshot, prior, sdir, collectedBlocks,
-              removedINodes, true);
-        }
-        priorDiff = diffList.getDiff(prior);
-        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
-          priorDiff.diff.destroyCreatedList(sdir, collectedBlocks,
-              removedINodes);
-        }
-      }
-      for (INode child : inode.asDirectory().getChildrenList(prior)) {
-        if (excludedNodes != null && excludedNodes.containsKey(child)) {
-          continue;
-        }
-        destroyDstSubtree(child, snapshot, prior, collectedBlocks,
-            removedINodes);
-      }
-    }
-  }
 }

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

@@ -25,14 +25,13 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
  * Feature for file with snapshot-related information.
  */
 @InterfaceAudience.Private
-public class FileWithSnapshotFeature extends INodeFile.Feature {
+public class FileWithSnapshotFeature implements INode.Feature {
   private final FileDiffList diffs;
   private boolean isCurrentFileDeleted = false;
   
@@ -57,10 +56,6 @@ public class FileWithSnapshotFeature extends INodeFile.Feature {
     isCurrentFileDeleted = true;
   }
 
-  public INodeFileAttributes getSnapshotINode(INodeFile f, Snapshot snapshot) {
-    return diffs.getSnapshotINode(snapshot, f);
-  }
-
   public FileDiffList getDiffs() {
     return diffs;
   }
@@ -90,7 +85,7 @@ public class FileWithSnapshotFeature extends INodeFile.Feature {
     if (snapshot == null) {
       // delete the current file while the file has snapshot feature
       if (!isCurrentFileDeleted()) {
-        file.recordModification(prior, null);
+        file.recordModification(prior);
         deleteCurrentFile();
       }
       collectBlocksAndClear(file, collectedBlocks, removedINodes);

+ 27 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java

@@ -44,6 +44,8 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
@@ -58,7 +60,7 @@ import com.google.common.primitives.SignedBytes;
  * by the namesystem and FSDirectory locks.
  */
 @InterfaceAudience.Private
-public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
+public class INodeDirectorySnapshottable extends INodeDirectory {
   /** Limit the number of snapshot per snapshottable directory. */
   static final int SNAPSHOT_LIMIT = 1 << 16;
 
@@ -115,8 +117,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
      * the two snapshots, while its associated value is a {@link ChildrenDiff}
      * storing the changes (creation/deletion) happened to the children (files).
      */
-    private final Map<INodeDirectoryWithSnapshot, ChildrenDiff> dirDiffMap = 
-        new HashMap<INodeDirectoryWithSnapshot, ChildrenDiff>();
+    private final Map<INodeDirectory, ChildrenDiff> dirDiffMap = 
+        new HashMap<INodeDirectory, ChildrenDiff>();
     
     SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot, Snapshot start,
         Snapshot end) {
@@ -126,8 +128,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     }
     
     /** Add a dir-diff pair */
-    private void addDirDiff(INodeDirectoryWithSnapshot dir,
-        byte[][] relativePath, ChildrenDiff diff) {
+    private void addDirDiff(INodeDirectory dir, byte[][] relativePath,
+        ChildrenDiff diff) {
       dirDiffMap.put(dir, diff);
       diffMap.put(dir, relativePath);
     }
@@ -154,8 +156,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
         if (node.isDirectory()) {
           ChildrenDiff dirDiff = dirDiffMap.get(node);
           List<DiffReportEntry> subList = dirDiff.generateReport(
-              diffMap.get(node), (INodeDirectoryWithSnapshot) node,
-              isFromEarlier());
+              diffMap.get(node), isFromEarlier());
           diffReportList.addAll(subList);
         }
       }
@@ -183,8 +184,11 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   private int snapshotQuota = SNAPSHOT_LIMIT;
 
   public INodeDirectorySnapshottable(INodeDirectory dir) {
-    super(dir, true, dir instanceof INodeDirectoryWithSnapshot ? 
-        ((INodeDirectoryWithSnapshot) dir).getDiffs(): null);
+    super(dir, true, true);
+    // add snapshot feature if the original directory does not have it
+    if (!isWithSnapshot()) {
+      addSnapshotFeature(null);
+    }
   }
   
   /** @return the number of existing snapshots. */
@@ -298,8 +302,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     snapshotsByNames.add(-i - 1, s);
 
     //set modification time
-    updateModificationTime(Time.now(), null, null);
-    s.getRoot().setModificationTime(getModificationTime(), null, null);
+    updateModificationTime(Time.now(), null);
+    s.getRoot().setModificationTime(getModificationTime(), null);
     return s;
   }
   
@@ -413,12 +417,12 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
     if (node.isDirectory()) {
       INodeDirectory dir = node.asDirectory();
-      if (dir instanceof INodeDirectoryWithSnapshot) {
-        INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
-        boolean change = sdir.computeDiffBetweenSnapshots(
-            diffReport.from, diffReport.to, diff);
+      DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
+      if (sf != null) {
+        boolean change = sf.computeDiffBetweenSnapshots(diffReport.from,
+            diffReport.to, diff, dir);
         if (change) {
-          diffReport.addDirDiff(sdir, relativePath, diff);
+          diffReport.addDirDiff(dir, relativePath, diff);
         }
       }
       ReadOnlyList<INode> children = dir.getChildrenList(diffReport
@@ -453,13 +457,15 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   INodeDirectory replaceSelf(final Snapshot latest, final INodeMap inodeMap)
       throws QuotaExceededException {
     if (latest == null) {
-      Preconditions.checkState(getLastSnapshot() == null,
+      Preconditions.checkState(
+          getDirectoryWithSnapshotFeature().getLastSnapshot() == null,
           "latest == null but getLastSnapshot() != null, this=%s", this);
-      return replaceSelf4INodeDirectory(inodeMap);
-    } else {
-      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
-          .recordModification(latest, null);
     }
+    INodeDirectory dir = replaceSelf4INodeDirectory(inodeMap);
+    if (latest != null) {
+      dir.recordModification(latest);
+    }
+    return dir;
   }
 
   @Override

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java

@@ -117,9 +117,8 @@ public class Snapshot implements Comparable<byte[]> {
     for(; inode != null; inode = inode.getParent()) {
       if (inode.isDirectory()) {
         final INodeDirectory dir = inode.asDirectory();
-        if (dir instanceof INodeDirectoryWithSnapshot) {
-          latest = ((INodeDirectoryWithSnapshot) dir).getDiffs().updatePrior(
-              anchor, latest);
+        if (dir.isWithSnapshot()) {
+          latest = dir.getDiffs().updatePrior(anchor, latest);
         }
       }
     }

+ 11 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java

@@ -36,8 +36,8 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -91,8 +91,7 @@ public class SnapshotFSImageFormat {
   public static void saveDirectoryDiffList(final INodeDirectory dir,
       final DataOutput out, final ReferenceMap referenceMap
       ) throws IOException {
-    saveINodeDiffs(dir instanceof INodeDirectoryWithSnapshot?
-        ((INodeDirectoryWithSnapshot)dir).getDiffs(): null, out, referenceMap);
+    saveINodeDiffs(dir.getDiffs(), out, referenceMap);
   }
   
   public static void saveFileDiffList(final INodeFile file,
@@ -139,7 +138,7 @@ public class SnapshotFSImageFormat {
    * @return The created node.
    */
   private static INode loadCreated(byte[] createdNodeName,
-      INodeDirectoryWithSnapshot parent) throws IOException {
+      INodeDirectory parent) throws IOException {
     // the INode in the created list should be a reference to another INode
     // in posterior SnapshotDiffs or one of the current children
     for (DirectoryDiff postDiff : parent.getDiffs()) {
@@ -165,7 +164,7 @@ public class SnapshotFSImageFormat {
    * @param in The {@link DataInput} to read.
    * @return The created list.
    */
-  private static List<INode> loadCreatedList(INodeDirectoryWithSnapshot parent,
+  private static List<INode> loadCreatedList(INodeDirectory parent,
       DataInput in) throws IOException {
     // read the size of the created list
     int createdSize = in.readInt();
@@ -188,7 +187,7 @@ public class SnapshotFSImageFormat {
    * @param loader The {@link Loader} instance.
    * @return The deleted list.
    */
-  private static List<INode> loadDeletedList(INodeDirectoryWithSnapshot parent,
+  private static List<INode> loadDeletedList(INodeDirectory parent,
       List<INode> createdList, DataInput in, FSImageFormat.Loader loader)
       throws IOException {
     int deletedSize = in.readInt();
@@ -239,11 +238,10 @@ public class SnapshotFSImageFormat {
   public static void loadDirectoryDiffList(INodeDirectory dir,
       DataInput in, FSImageFormat.Loader loader) throws IOException {
     final int size = in.readInt();
-    if (dir instanceof INodeDirectoryWithSnapshot) {
-      INodeDirectoryWithSnapshot withSnapshot = (INodeDirectoryWithSnapshot)dir;
-      DirectoryDiffList diffs = withSnapshot.getDiffs();
+    if (dir.isWithSnapshot()) {
+      DirectoryDiffList diffs = dir.getDiffs();
       for (int i = 0; i < size; i++) {
-        diffs.addFirst(loadDirectoryDiff(withSnapshot, in, loader));
+        diffs.addFirst(loadDirectoryDiff(dir, in, loader));
       }
     }
   }
@@ -277,9 +275,8 @@ public class SnapshotFSImageFormat {
    *               using.
    * @return A {@link DirectoryDiff}.
    */
-  private static DirectoryDiff loadDirectoryDiff(
-      INodeDirectoryWithSnapshot parent, DataInput in,
-      FSImageFormat.Loader loader) throws IOException {
+  private static DirectoryDiff loadDirectoryDiff(INodeDirectory parent,
+      DataInput in, FSImageFormat.Loader loader) throws IOException {
     // 1. Read the full path of the Snapshot root to identify the Snapshot
     final Snapshot snapshot = loader.getSnapshot(in);
 

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

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

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

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

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

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

+ 10 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.util.VersionInfo;
 
 import java.io.*;
 import java.net.*;
+import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.*;
 import java.util.concurrent.TimeoutException;
@@ -1036,20 +1037,20 @@ public class DFSTestUtil {
     // OP_ADD_CACHE_POOL
     filesystem.addCachePool(new CachePoolInfo("pool1"));
     // OP_MODIFY_CACHE_POOL
-    filesystem.modifyCachePool(new CachePoolInfo("pool1").setWeight(99));
+    filesystem.modifyCachePool(new CachePoolInfo("pool1").setLimit(99l));
     // OP_ADD_PATH_BASED_CACHE_DIRECTIVE
     long id = filesystem.addCacheDirective(
         new CacheDirectiveInfo.Builder().
             setPath(new Path("/path")).
             setReplication((short)1).
             setPool("pool1").
-            build());
+            build(), EnumSet.of(CacheFlag.FORCE));
     // OP_MODIFY_PATH_BASED_CACHE_DIRECTIVE
     filesystem.modifyCacheDirective(
         new CacheDirectiveInfo.Builder().
             setId(id).
             setReplication((short)2).
-            build());
+            build(), EnumSet.of(CacheFlag.FORCE));
     // OP_REMOVE_PATH_BASED_CACHE_DIRECTIVE
     filesystem.removeCacheDirective(id);
     // OP_REMOVE_CACHE_POOL
@@ -1059,4 +1060,10 @@ public class DFSTestUtil {
   public static void abortStream(DFSOutputStream out) throws IOException {
     out.abort();
   }
+
+  public static byte[] asArray(ByteBuffer buf) {
+    byte arr[] = new byte[buf.remaining()];
+    buf.duplicate().get(arr);
+    return arr;
+  }
 }

+ 3 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -2143,17 +2143,14 @@ public class MiniDFSCluster {
   }
 
   /**
-   * Get a storage directory for a datanode. There are two storage directories
-   * per datanode:
+   * Get a storage directory for a datanode.
    * <ol>
    * <li><base directory>/data/data<2*dnIndex + 1></li>
    * <li><base directory>/data/data<2*dnIndex + 2></li>
    * </ol>
    *
    * @param dnIndex datanode index (starts from 0)
-   * @param dirIndex directory index (0 or 1). Index 0 provides access to the
-   *          first storage directory. Index 1 provides access to the second
-   *          storage directory.
+   * @param dirIndex directory index.
    * @return Storage directory
    */
   public static File getStorageDir(int dnIndex, int dirIndex) {
@@ -2164,7 +2161,7 @@ public class MiniDFSCluster {
    * Calculate the DN instance-specific path for appending to the base dir
    * to determine the location of the storage of a DN instance in the mini cluster
    * @param dnIndex datanode index
-   * @param dirIndex directory index (0 or 1).
+   * @param dirIndex directory index.
    * @return
    */
   private static String getStorageDirPath(int dnIndex, int dirIndex) {

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

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

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

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

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

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

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

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

+ 135 - 36
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java

@@ -71,7 +71,15 @@ import org.mockito.invocation.InvocationOnMock;
 /**
  * This test simulates a variety of situations when blocks are being
  * intentionally corrupted, unexpectedly modified, and so on before a block
- * report is happening
+ * report is happening.
+ *
+ * For each test case it runs two variations:
+ *  #1 - For a given DN, the first variation sends block reports for all
+ *       storages in a single call to the NN.
+ *  #2 - For a given DN, the second variation sends block reports for each
+ *       storage in a separate call.
+ *
+ * The behavior should be the same in either variation.
  */
 public class TestBlockReport {
   public static final Log LOG = LogFactory.getLog(TestBlockReport.class);
@@ -157,6 +165,113 @@ public class TestBlockReport {
     return reports;
   }
 
+  /**
+   * Utility routine to send block reports to the NN, either in a single call
+   * or reporting one storage per call.
+   *
+   * @param dnR
+   * @param poolId
+   * @param reports
+   * @param needtoSplit
+   * @throws IOException
+   */
+  private void sendBlockReports(DatanodeRegistration dnR, String poolId,
+      StorageBlockReport[] reports, boolean needtoSplit) throws IOException {
+    if (!needtoSplit) {
+      LOG.info("Sending combined block reports for " + dnR);
+      cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    } else {
+      for (StorageBlockReport report : reports) {
+        LOG.info("Sending block report for storage " + report.getStorage().getStorageID());
+        StorageBlockReport[] singletonReport = { report };
+        cluster.getNameNodeRpc().blockReport(dnR, poolId, singletonReport);
+      }
+    }
+  }
+
+  /**
+   * Test variations blockReport_01 through blockReport_09 with combined
+   * and split block reports.
+   */
+  @Test
+  public void blockReportCombined_01() throws IOException {
+    blockReport_01(false);
+  }
+
+  @Test
+  public void blockReportSplit_01() throws IOException {
+    blockReport_01(true);
+  }
+
+  @Test
+  public void blockReportCombined_02() throws IOException {
+    blockReport_02(false);
+  }
+
+  @Test
+  public void blockReportSplit_02() throws IOException {
+    blockReport_02(true);
+  }
+
+  @Test
+  public void blockReportCombined_03() throws IOException {
+    blockReport_03(false);
+  }
+
+  @Test
+  public void blockReportSplit_03() throws IOException {
+    blockReport_03(true);
+  }
+
+  @Test
+  public void blockReportCombined_04() throws IOException {
+    blockReport_04(false);
+  }
+
+  @Test
+  public void blockReportSplit_04() throws IOException {
+    blockReport_04(true);
+  }
+
+  @Test
+  public void blockReportCombined_06() throws Exception {
+    blockReport_06(false);
+  }
+
+  @Test
+  public void blockReportSplit_06() throws Exception {
+    blockReport_06(true);
+  }
+
+  @Test
+  public void blockReportCombined_07() throws Exception {
+    blockReport_07(false);
+  }
+
+  @Test
+  public void blockReportSplit_07() throws Exception {
+    blockReport_07(true);
+  }
+
+  @Test
+  public void blockReportCombined_08() throws Exception {
+    blockReport_08(false);
+  }
+
+  @Test
+  public void blockReportSplit_08() throws Exception {
+    blockReport_08(true);
+  }
+
+  @Test
+  public void blockReportCombined_09() throws Exception {
+    blockReport_09(false);
+  }
+
+  @Test
+  public void blockReportSplit_09() throws Exception {
+    blockReport_09(true);
+  }
   /**
    * Test write a file, verifies and closes it. Then the length of the blocks
    * are messed up and BlockReport is forced.
@@ -164,8 +279,7 @@ public class TestBlockReport {
    *
    * @throws java.io.IOException on an error
    */
-  @Test
-  public void blockReport_01() throws IOException {
+  private void blockReport_01(boolean splitBlockReports) throws IOException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
 
@@ -198,7 +312,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
 
     List<LocatedBlock> blocksAfterReport =
       DFSTestUtil.getAllBlocks(fs.open(filePath));
@@ -224,8 +338,7 @@ public class TestBlockReport {
    *
    * @throws IOException in case of errors
    */
-  @Test
-  public void blockReport_02() throws IOException {
+  private void blockReport_02(boolean splitBlockReports) throws IOException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     LOG.info("Running test " + METHOD_NAME);
 
@@ -280,7 +393,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn0.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn0, poolId, false, false);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
 
     BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
         .getBlockManager());
@@ -301,8 +414,7 @@ public class TestBlockReport {
    *
    * @throws IOException in case of an error
    */
-  @Test
-  public void blockReport_03() throws IOException {
+  private void blockReport_03(boolean splitBlockReports) throws IOException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     ArrayList<Block> blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath);
@@ -312,11 +424,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
-    DatanodeCommand dnCmd =
-      cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Got the command: " + dnCmd);
-    }
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
     printStats();
 
     assertThat("Wrong number of corrupt blocks",
@@ -333,8 +441,7 @@ public class TestBlockReport {
    *
    * @throws IOException in case of an error
    */
-  @Test
-  public void blockReport_04() throws IOException {
+  private void blockReport_04(boolean splitBlockReports) throws IOException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     DFSTestUtil.createFile(fs, filePath,
@@ -352,11 +459,7 @@ public class TestBlockReport {
 
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
-    DatanodeCommand dnCmd =
-        cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Got the command: " + dnCmd);
-    }
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
     printStats();
 
     assertThat("Wrong number of corrupt blocks",
@@ -373,8 +476,7 @@ public class TestBlockReport {
    *
    * @throws IOException in case of an error
    */
-  @Test
-  public void blockReport_06() throws Exception {
+  private void blockReport_06(boolean splitBlockReports) throws Exception {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     final int DN_N1 = DN_N0 + 1;
@@ -387,7 +489,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
     printStats();
     assertEquals("Wrong number of PendingReplication Blocks",
       0, cluster.getNamesystem().getUnderReplicatedBlocks());
@@ -406,8 +508,7 @@ public class TestBlockReport {
    *
    * @throws IOException in case of an error
    */
-  @Test
-  public void blockReport_07() throws Exception {
+  private void blockReport_07(boolean splitBlockReports) throws Exception {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     final int DN_N1 = DN_N0 + 1;
@@ -421,7 +522,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
     printStats();
 
     assertThat("Wrong number of corrupt blocks",
@@ -432,7 +533,7 @@ public class TestBlockReport {
                cluster.getNamesystem().getPendingReplicationBlocks(), is(0L));
 
     reports = getBlockReports(dn, poolId, true, true);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
     printStats();
 
     assertThat("Wrong number of corrupt blocks",
@@ -458,8 +559,7 @@ public class TestBlockReport {
    *
    * @throws IOException in case of an error
    */
-  @Test
-  public void blockReport_08() throws IOException {
+  private void blockReport_08(boolean splitBlockReports) throws IOException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     final int DN_N1 = DN_N0 + 1;
@@ -483,8 +583,8 @@ public class TestBlockReport {
       DataNode dn = cluster.getDataNodes().get(DN_N1);
       String poolId = cluster.getNamesystem().getBlockPoolId();
       DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
-      StorageBlockReport[] report = getBlockReports(dn, poolId, false, false);
-      cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
+      StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
+      sendBlockReports(dnR, poolId, reports, splitBlockReports);
       printStats();
       assertEquals("Wrong number of PendingReplication blocks",
         blocks.size(), cluster.getNamesystem().getPendingReplicationBlocks());
@@ -500,8 +600,7 @@ public class TestBlockReport {
   // Similar to BlockReport_08 but corrupts GS and len of the TEMPORARY's
   // replica block. Expect the same behaviour: NN should simply ignore this
   // block
-  @Test
-  public void blockReport_09() throws IOException {
+  private void blockReport_09(boolean splitBlockReports) throws IOException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     final int DN_N1 = DN_N0 + 1;
@@ -526,8 +625,8 @@ public class TestBlockReport {
       DataNode dn = cluster.getDataNodes().get(DN_N1);
       String poolId = cluster.getNamesystem().getBlockPoolId();
       DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
-      StorageBlockReport[] report = getBlockReports(dn, poolId, true, true);
-      cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
+      StorageBlockReport[] reports = getBlockReports(dn, poolId, true, true);
+      sendBlockReports(dnR, poolId, reports, splitBlockReports);
       printStats();
       assertEquals("Wrong number of PendingReplication blocks",
         2, cluster.getNamesystem().getPendingReplicationBlocks());

+ 87 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java

@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.*;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.junit.Test;
+
+import static org.hamcrest.core.IsNot.not;
+import static org.junit.Assert.*;
+
+
+/**
+ * Test to verify that the DataNode Uuid is correctly initialized before
+ * FsDataSet initialization.
+ */
+public class TestDataNodeInitStorage {
+  public static final Log LOG = LogFactory.getLog(TestDataNodeInitStorage.class);
+
+  static private class SimulatedFsDatasetVerifier extends SimulatedFSDataset {
+    static class Factory extends FsDatasetSpi.Factory<SimulatedFSDataset> {
+      @Override
+      public SimulatedFsDatasetVerifier newInstance(
+          DataNode datanode, DataStorage storage,
+          Configuration conf) throws IOException {
+        return new SimulatedFsDatasetVerifier(storage, conf);
+      }
+
+      @Override
+      public boolean isSimulated() {
+        return true;
+      }
+    }
+
+    public static void setFactory(Configuration conf) {
+      conf.set(DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
+               Factory.class.getName());
+    }
+
+    // This constructor does the actual verification by ensuring that
+    // the DatanodeUuid is initialized.
+    public SimulatedFsDatasetVerifier(DataStorage storage, Configuration conf) {
+      super(storage, conf);
+      LOG.info("Assigned DatanodeUuid is " + storage.getDatanodeUuid());
+      assert(storage.getDatanodeUuid() != null);
+      assert(storage.getDatanodeUuid().length() != 0);
+    }
+  }
+
+
+  @Test (timeout = 60000)
+  public void testDataNodeInitStorage() throws Throwable {
+    // Create configuration to use SimulatedFsDatasetVerifier#Factory.
+    Configuration conf = new HdfsConfiguration();
+    SimulatedFsDatasetVerifier.setFactory(conf);
+
+    // Start a cluster so that SimulatedFsDatasetVerifier constructor is
+    // invoked.
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    cluster.shutdown();
+  }
+}

+ 213 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java

@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
+import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.*;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * This test verifies that incremental block reports from a single DataNode are
+ * correctly handled by NN. Tests the following variations:
+ *  #1 - Incremental BRs from all storages combined in a single call.
+ *  #2 - Incremental BRs from separate storages sent in separate calls.
+ *
+ *  We also verify that the DataNode is not splitting the reports (it may do so
+ *  in the future).
+ */
+public class TestIncrementalBrVariations {
+  public static final Log LOG = LogFactory.getLog(TestIncrementalBrVariations.class);
+
+  private static short NUM_DATANODES = 1;
+  static final int BLOCK_SIZE = 1024;
+  static final int NUM_BLOCKS = 10;
+  private static final long seed = 0xFACEFEEDL;
+  private static final String NN_METRICS = "NameNodeActivity";
+
+
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private DFSClient client;
+  private static Configuration conf;
+
+  static {
+    ((Log4JLogger) NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger) BlockManager.blockLog).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger) NameNode.blockStateChangeLog).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger) LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger) DataNode.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger) TestIncrementalBrVariations.LOG).getLogger().setLevel(Level.ALL);
+  }
+
+  @Before
+  public void startUpCluster() throws IOException {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
+    fs = cluster.getFileSystem();
+    client = new DFSClient(new InetSocketAddress("localhost", cluster.getNameNodePort()),
+                           cluster.getConfiguration(0));
+  }
+
+  @After
+  public void shutDownCluster() throws IOException {
+    client.close();
+    fs.close();
+    cluster.shutdownDataNodes();
+    cluster.shutdown();
+  }
+
+  /**
+   * Incremental BRs from all storages combined in a single message.
+   */
+  @Test
+  public void testCombinedIncrementalBlockReport() throws IOException {
+    verifyIncrementalBlockReports(false);
+  }
+
+  /**
+   * One incremental BR per storage.
+   */
+  @Test
+  public void testSplitIncrementalBlockReport() throws IOException {
+    verifyIncrementalBlockReports(true);
+  }
+
+  private LocatedBlocks createFileGetBlocks(String filenamePrefix) throws IOException {
+    Path filePath = new Path("/" + filenamePrefix + ".dat");
+
+    // Write out a file with a few blocks, get block locations.
+    DFSTestUtil.createFile(fs, filePath, BLOCK_SIZE, BLOCK_SIZE * NUM_BLOCKS,
+                           BLOCK_SIZE, NUM_DATANODES, seed);
+
+    // Get the block list for the file with the block locations.
+    LocatedBlocks blocks = client.getLocatedBlocks(
+        filePath.toString(), 0, BLOCK_SIZE * NUM_BLOCKS);
+    assertThat(cluster.getNamesystem().getUnderReplicatedBlocks(), is(0L));
+    return blocks;
+  }
+
+  public void verifyIncrementalBlockReports(boolean splitReports) throws IOException {
+    // Get the block list for the file with the block locations.
+    LocatedBlocks blocks = createFileGetBlocks(GenericTestUtils.getMethodName());
+
+    // A blocks belong to the same file, hence same BP
+    DataNode dn = cluster.getDataNodes().get(0);
+    String poolId = cluster.getNamesystem().getBlockPoolId();
+    DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
+
+    // We will send 'fake' incremental block reports to the NN that look
+    // like they originated from DN 0.
+    StorageReceivedDeletedBlocks reports[] =
+        new StorageReceivedDeletedBlocks[dn.getFSDataset().getVolumes().size()];
+
+    // Lie to the NN that one block on each storage has been deleted.
+    for (int i = 0; i < reports.length; ++i) {
+      FsVolumeSpi volume = dn.getFSDataset().getVolumes().get(i);
+
+      boolean foundBlockOnStorage = false;
+      ReceivedDeletedBlockInfo rdbi[] = new ReceivedDeletedBlockInfo[1];
+
+      // Find the first block on this storage and mark it as deleted for the
+      // report.
+      for (LocatedBlock block : blocks.getLocatedBlocks()) {
+        if (block.getStorageIDs()[0].equals(volume.getStorageID())) {
+          rdbi[0] = new ReceivedDeletedBlockInfo(block.getBlock().getLocalBlock(),
+              ReceivedDeletedBlockInfo.BlockStatus.DELETED_BLOCK, null);
+          foundBlockOnStorage = true;
+          break;
+        }
+      }
+
+      assertTrue(foundBlockOnStorage);
+      reports[i] = new StorageReceivedDeletedBlocks(volume.getStorageID(), rdbi);
+
+      if (splitReports) {
+        // If we are splitting reports then send the report for this storage now.
+        StorageReceivedDeletedBlocks singletonReport[] = { reports[i] };
+        cluster.getNameNodeRpc().blockReceivedAndDeleted(dnR, poolId, singletonReport);
+      }
+    }
+
+    if (!splitReports) {
+      // Send a combined report.
+      cluster.getNameNodeRpc().blockReceivedAndDeleted(dnR, poolId, reports);
+    }
+
+    // Make sure that the deleted block from each storage was picked up
+    // by the NameNode.
+    assertThat(cluster.getNamesystem().getMissingBlocksCount(), is((long) reports.length));
+  }
+
+  /**
+   * Verify that the DataNode sends a single incremental block report for all
+   * storages.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test (timeout=60000)
+  public void testDataNodeDoesNotSplitReports()
+      throws IOException, InterruptedException {
+    LocatedBlocks blocks = createFileGetBlocks(GenericTestUtils.getMethodName());
+    assertThat(cluster.getDataNodes().size(), is(1));
+    DataNode dn = cluster.getDataNodes().get(0);
+
+    // Remove all blocks from the DataNode.
+    for (LocatedBlock block : blocks.getLocatedBlocks()) {
+      dn.notifyNamenodeDeletedBlock(
+          block.getBlock(), block.getStorageIDs()[0]);
+    }
+
+    LOG.info("Triggering report after deleting blocks");
+    long ops = getLongCounter("BlockReceivedAndDeletedOps", getMetrics(NN_METRICS));
+
+    // Trigger a report to the NameNode and give it a few seconds.
+    DataNodeTestUtils.triggerBlockReport(dn);
+    Thread.sleep(5000);
+
+    // Ensure that NameNodeRpcServer.blockReceivedAndDeletes is invoked
+    // exactly once after we triggered the report.
+    assertCounter("BlockReceivedAndDeletedOps", ops+1, getMetrics(NN_METRICS));
+  }
+}

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

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

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

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

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

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

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

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

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

@@ -41,8 +41,8 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.log4j.Level;

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -780,7 +781,7 @@ public class TestINodeFile {
       }
       System.out.println("Adding component " + DFSUtil.bytes2String(component));
       dir = new INodeDirectory(++id, component, permstatus, 0);
-      prev.addChild(dir, false, null, null);
+      prev.addChild(dir, false, null);
       prev = dir;
     }
     return dir; // Last Inode in the chain
@@ -921,6 +922,7 @@ public class TestINodeFile {
   public void testDotdotInodePath() throws Exception {
     final Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
+    DFSClient client = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       cluster.waitActive();
@@ -933,7 +935,7 @@ public class TestINodeFile {
       long parentId = fsdir.getINode("/").getId();
       String testPath = "/.reserved/.inodes/" + dirId + "/..";
 
-      DFSClient client = new DFSClient(NameNode.getAddress(conf), conf);
+      client = new DFSClient(NameNode.getAddress(conf), conf);
       HdfsFileStatus status = client.getFileInfo(testPath);
       assertTrue(parentId == status.getFileId());
       
@@ -943,6 +945,7 @@ public class TestINodeFile {
       assertTrue(parentId == status.getFileId());
       
     } finally {
+      IOUtils.cleanup(LOG, client);
       if (cluster != null) {
         cluster.shutdown();
       }

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

@@ -31,7 +31,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -206,8 +205,7 @@ public class TestSnapshotPathINodes {
     // Check the INode for file1 (snapshot file)
     INode snapshotFileNode = inodes[inodes.length - 1]; 
     assertINodeFile(snapshotFileNode, file1);
-    assertTrue(snapshotFileNode.getParent() instanceof 
-        INodeDirectoryWithSnapshot);
+    assertTrue(snapshotFileNode.getParent().isWithSnapshot());
     
     // Call getExistingPathINodes and request only one INode.
     nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 1, false);

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

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

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

@@ -44,7 +44,7 @@ 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.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Before;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java

@@ -358,7 +358,7 @@ public class TestNestedSnapshots {
     
     FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
     INode subNode = fsdir.getINode(sub.toString());
-    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
+    assertTrue(subNode.asDirectory().isWithSnapshot());
     
     hdfs.allowSnapshot(sub);
     subNode = fsdir.getINode(sub.toString());
@@ -366,6 +366,6 @@ public class TestNestedSnapshots {
     
     hdfs.disallowSnapshot(sub);
     subNode = fsdir.getINode(sub.toString());
-    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
+    assertTrue(subNode.asDirectory().isWithSnapshot());
   }
 }

+ 45 - 58
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -59,12 +59,11 @@ 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.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.ChildrenDiff;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -757,10 +756,10 @@ public class TestRenameWithSnapshots {
     // only 2 references: one in deleted list of sdir1, one in created list of
     // sdir1
     assertEquals(2, fooWithCount.getReferenceCount());
-    INodeDirectoryWithSnapshot foo = (INodeDirectoryWithSnapshot) fooWithCount
-        .asDirectory();
+    INodeDirectory foo = fooWithCount.asDirectory();
     assertEquals(1, foo.getDiffs().asList().size());
-    assertEquals("s1", foo.getLastSnapshot().getRoot().getLocalName());
+    assertEquals("s1", foo.getDirectoryWithSnapshotFeature().getLastSnapshot()
+        .getRoot().getLocalName());
     INodeFile bar1 = fsdir.getINode4Write(bar1_dir1.toString()).asFile();
     assertEquals(1, bar1.getDiffs().asList().size());
     assertEquals("s1", bar1.getDiffs().getLastSnapshot().getRoot()
@@ -973,8 +972,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount fooWithCount = (WithCount) fooRef.getReferredINode();
     // 5 references: s1, s22, s333, s2222, current tree of sdir1
     assertEquals(5, fooWithCount.getReferenceCount());
-    INodeDirectoryWithSnapshot foo = (INodeDirectoryWithSnapshot) fooWithCount
-        .asDirectory();
+    INodeDirectory foo = fooWithCount.asDirectory();
     List<DirectoryDiff> fooDiffs = foo.getDiffs().asList();
     assertEquals(4, fooDiffs.size());
     assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1032,7 +1030,7 @@ public class TestRenameWithSnapshots {
     fooRef = fsdir.getINode(foo_s2222.toString()).asReference();
     fooWithCount = (WithCount) fooRef.getReferredINode();
     assertEquals(4, fooWithCount.getReferenceCount());
-    foo = (INodeDirectoryWithSnapshot) fooWithCount.asDirectory();
+    foo = fooWithCount.asDirectory();
     fooDiffs = foo.getDiffs().asList();
     assertEquals(4, fooDiffs.size());
     assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1171,8 +1169,7 @@ public class TestRenameWithSnapshots {
     assertTrue(fooRef instanceof INodeReference.WithName);
     INodeReference.WithCount fooWC = (WithCount) fooRef.getReferredINode();
     assertEquals(1, fooWC.getReferenceCount());
-    INodeDirectoryWithSnapshot fooDir = (INodeDirectoryWithSnapshot) fooWC
-        .getReferredINode().asDirectory();
+    INodeDirectory fooDir = fooWC.getReferredINode().asDirectory();
     List<DirectoryDiff> diffs = fooDir.getDiffs().asList();
     assertEquals(1, diffs.size());
     assertEquals("s2", diffs.get(0).snapshot.getRoot().getLocalName());
@@ -1263,7 +1260,7 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory mockDir2 = spy(dir2);
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject(), (INodeMap) anyObject());
+            (Snapshot) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     
@@ -1288,9 +1285,8 @@ public class TestRenameWithSnapshots {
     assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
     
     INode fooNode = fsdir.getINode4Write(foo.toString());
-    assertTrue(fooNode instanceof INodeDirectoryWithSnapshot);
-    List<DirectoryDiff> fooDiffs = ((INodeDirectoryWithSnapshot) fooNode)
-        .getDiffs().asList();
+    assertTrue(fooNode.isDirectory() && fooNode.asDirectory().isWithSnapshot());
+    List<DirectoryDiff> fooDiffs = fooNode.asDirectory().getDiffs().asList();
     assertEquals(1, fooDiffs.size());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     
@@ -1302,7 +1298,7 @@ public class TestRenameWithSnapshots {
     assertFalse(hdfs.exists(newfoo));
     INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
-    assertFalse(dir2Node instanceof INodeDirectoryWithSnapshot);
+    assertFalse(dir2Node.isWithSnapshot());
     ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
     assertEquals(1, dir2Children.size());
     assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
@@ -1331,7 +1327,7 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory mockDir2 = spy(dir2);
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject(), (INodeMap) anyObject());
+            (Snapshot) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     
@@ -1366,7 +1362,7 @@ public class TestRenameWithSnapshots {
     assertFalse(hdfs.exists(newfoo));
     INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
-    assertFalse(dir2Node instanceof INodeDirectoryWithSnapshot);
+    assertFalse(dir2Node.isWithSnapshot());
     ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
     assertEquals(1, dir2Children.size());
     assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
@@ -1393,7 +1389,7 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
     INodeDirectory mockDir3 = spy(dir3);
     doReturn(false).when(mockDir3).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject(), (INodeMap) anyObject());
+            (Snapshot) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     
@@ -1420,8 +1416,7 @@ public class TestRenameWithSnapshots {
     INode fooNode = fsdir.getINode4Write(foo_dir2.toString());
     assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
     assertTrue(fooNode instanceof INodeReference.DstReference);
-    List<DirectoryDiff> fooDiffs = ((INodeDirectoryWithSnapshot) fooNode
-        .asDirectory()).getDiffs().asList();
+    List<DirectoryDiff> fooDiffs = fooNode.asDirectory().getDiffs().asList();
     assertEquals(1, fooDiffs.size());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     
@@ -1455,8 +1450,7 @@ public class TestRenameWithSnapshots {
     assertTrue(hdfs.exists(foo_s3));
     
     assertTrue(fooNode instanceof INodeReference.DstReference);
-    fooDiffs = ((INodeDirectoryWithSnapshot) fooNode.asDirectory()).getDiffs()
-        .asList();
+    fooDiffs = fooNode.asDirectory().getDiffs().asList();
     assertEquals(2, fooDiffs.size());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     assertEquals("s3", fooDiffs.get(1).snapshot.getRoot().getLocalName());
@@ -1495,10 +1489,9 @@ public class TestRenameWithSnapshots {
     INodeDirectory mockDir3 = spy(dir3);
     // fail the rename but succeed in undo
     doReturn(false).when(mockDir3).addChild((INode) Mockito.isNull(),
-        anyBoolean(), (Snapshot) anyObject(), (INodeMap) anyObject());
-    Mockito.when(mockDir3.addChild((INode) Mockito.isNotNull(), 
-        anyBoolean(), (Snapshot) anyObject(), 
-        (INodeMap) anyObject())).thenReturn(false).thenCallRealMethod();
+        anyBoolean(), (Snapshot) anyObject());
+    Mockito.when(mockDir3.addChild((INode) Mockito.isNotNull(), anyBoolean(), 
+        (Snapshot) anyObject())).thenReturn(false).thenCallRealMethod();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     foo3Node.setParent(mockDir3);
@@ -1561,7 +1554,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(null));
     assertEquals(1, childrenList.size());
     INode fooNode = childrenList.get(0);
-    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    assertTrue(fooNode.asDirectory().isWithSnapshot());
     INode barNode = fsdir.getINode4Write(bar.toString());
     assertTrue(barNode.getClass() == INodeFile.class);
     assertSame(fooNode, barNode.getParent());
@@ -1637,7 +1630,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(null));
     assertEquals(1, childrenList.size());
     INode fooNode = childrenList.get(0);
-    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    assertTrue(fooNode.asDirectory().isWithSnapshot());
     assertSame(dir1Node, fooNode.getParent());
     List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
         .getDiffs().asList();
@@ -1656,7 +1649,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(null));
     assertEquals(1, childrenList.size());
     INode subdir2Node = childrenList.get(0);
-    assertTrue(subdir2Node.getClass() == INodeDirectoryWithSnapshot.class);
+    assertTrue(subdir2Node.asDirectory().isWithSnapshot());
     assertSame(dir2Node, subdir2Node.getParent());
     assertSame(subdir2Node, fsdir.getINode4Write(sub_dir2.toString()));
     INode subsubdir2Node = fsdir.getINode4Write(subsub_dir2.toString());
@@ -1669,7 +1662,7 @@ public class TestRenameWithSnapshots {
     assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
     assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
     
-    diffList = ((INodeDirectoryWithSnapshot) subdir2Node).getDiffs().asList();
+    diffList = subdir2Node.asDirectory().getDiffs().asList();
     assertEquals(0, diffList.size());
   }
   
@@ -1697,8 +1690,7 @@ public class TestRenameWithSnapshots {
     }
     
     // check
-    INodeDirectoryWithSnapshot fooNode = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(foo.toString());
+    INodeDirectory fooNode = fsdir.getINode4Write(foo.toString()).asDirectory();
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     assertEquals(1, children.size());
     List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
@@ -1948,8 +1940,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount wc = 
         (WithCount) fooRef.asReference().getReferredINode();
     assertEquals(1, wc.getReferenceCount());
-    INodeDirectoryWithSnapshot fooNode = 
-        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    INodeDirectory fooNode = wc.getReferredINode().asDirectory();
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     assertEquals(1, children.size());
     assertEquals(bar.getName(), children.get(0).getLocalName());
@@ -2017,8 +2008,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount wc = 
         (WithCount) fooRef.asReference().getReferredINode();
     assertEquals(2, wc.getReferenceCount());
-    INodeDirectoryWithSnapshot fooNode = 
-        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    INodeDirectory fooNode = wc.getReferredINode().asDirectory();
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     assertEquals(3, children.size());
     assertEquals(bar.getName(), children.get(0).getLocalName());
@@ -2044,9 +2034,9 @@ public class TestRenameWithSnapshots {
   
   /**
    * This test demonstrates that 
-   * {@link INodeDirectoryWithSnapshot#removeChild(INode, Snapshot, INodeMap)}
+   * {@link INodeDirectory#removeChild(INode, Snapshot)}
    * and 
-   * {@link INodeDirectoryWithSnapshot#addChild(INode, boolean, Snapshot, INodeMap)}
+   * {@link INodeDirectory#addChild(INode, boolean, Snapshot)}
    * should use {@link INode#isInLatestSnapshot(Snapshot)} to check if the 
    * added/removed child should be recorded in snapshots.
    */
@@ -2063,7 +2053,7 @@ public class TestRenameWithSnapshots {
     hdfs.mkdirs(foo);
     SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
     final Path bar = new Path(foo, "bar");
-    // create file bar, and foo will become an INodeDirectoryWithSnapshot
+    // create file bar, and foo will become an INodeDirectory with snapshot
     DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
     // delete snapshot s1. now foo is not in any snapshot
     hdfs.deleteSnapshot(dir1, "s1");
@@ -2079,7 +2069,7 @@ public class TestRenameWithSnapshots {
     
     // delete /dir2/foo. Since it is not in any snapshot, we will call its 
     // destroy function. If we do not use isInLatestSnapshot in removeChild and
-    // addChild methods in INodeDirectoryWithSnapshot, the file bar will be 
+    // addChild methods in INodeDirectory (with snapshot), the file bar will be 
     // stored in the deleted list of foo, and will be destroyed.
     hdfs.delete(foo2, true);
     
@@ -2130,8 +2120,8 @@ public class TestRenameWithSnapshots {
     // check the internal
     assertFalse("after deleting s0, " + foo_s0 + " should not exist",
         hdfs.exists(foo_s0));
-    INodeDirectoryWithSnapshot dir2Node = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(dir2.toString());
+    INodeDirectory dir2Node = fsdir.getINode4Write(dir2.toString())
+        .asDirectory();
     assertTrue("the diff list of " + dir2
         + " should be empty after deleting s0", dir2Node.getDiffs().asList()
         .isEmpty());
@@ -2140,16 +2130,14 @@ public class TestRenameWithSnapshots {
     INode fooRefNode = fsdir.getINode4Write(newfoo.toString());
     assertTrue(fooRefNode instanceof INodeReference.DstReference);
     INodeDirectory fooNode = fooRefNode.asDirectory();
-    // fooNode should be still INodeDirectoryWithSnapshot since we call
+    // fooNode should be still INodeDirectory (With Snapshot) since we call
     // recordModification before the rename
-    assertTrue(fooNode instanceof INodeDirectoryWithSnapshot);
-    assertTrue(((INodeDirectoryWithSnapshot) fooNode).getDiffs().asList()
-        .isEmpty());
+    assertTrue(fooNode.isWithSnapshot());
+    assertTrue(fooNode.getDiffs().asList().isEmpty());
     INodeDirectory barNode = fooNode.getChildrenList(null).get(0).asDirectory();
-    // bar should also be an INodeDirectoryWithSnapshot, and both of its diff 
+    // bar should also be INodeDirectory (With Snapshot), and both of its diff 
     // list and children list are empty 
-    assertTrue(((INodeDirectoryWithSnapshot) barNode).getDiffs().asList()
-        .isEmpty());
+    assertTrue(barNode.getDiffs().asList().isEmpty());
     assertTrue(barNode.getChildrenList(null).isEmpty());
     
     restartClusterAndCheckImage(true);
@@ -2199,8 +2187,8 @@ public class TestRenameWithSnapshots {
     assertTrue(hdfs.exists(file_s0));
     
     // check dir1: foo should be in the created list of s0
-    INodeDirectoryWithSnapshot dir1Node = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(dir1.toString());
+    INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
+        .asDirectory();
     List<DirectoryDiff> dir1DiffList = dir1Node.getDiffs().asList();
     assertEquals(1, dir1DiffList.size());
     List<INode> dList = dir1DiffList.get(0).getChildrenDiff()
@@ -2215,8 +2203,8 @@ public class TestRenameWithSnapshots {
     
     // check foo and its subtree
     final Path newbar = new Path(newfoo, bar.getName());
-    INodeDirectoryWithSnapshot barNode = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(newbar.toString());
+    INodeDirectory barNode = fsdir.getINode4Write(newbar.toString())
+        .asDirectory();
     assertSame(fooNode.asDirectory(), barNode.getParent());
     // bar should only have a snapshot diff for s0
     List<DirectoryDiff> barDiffList = barNode.getDiffs().asList();
@@ -2229,8 +2217,8 @@ public class TestRenameWithSnapshots {
     
     // check dir2: a WithName instance for foo should be in the deleted list
     // of the snapshot diff for s2
-    INodeDirectoryWithSnapshot dir2Node = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(dir2.toString());
+    INodeDirectory dir2Node = fsdir.getINode4Write(dir2.toString())
+        .asDirectory();
     List<DirectoryDiff> dir2DiffList = dir2Node.getDiffs().asList();
     // dir2Node should contain 2 snapshot diffs, one for s2, and the other was
     // originally s1 (created when dir2 was transformed to a snapshottable dir),
@@ -2287,8 +2275,7 @@ public class TestRenameWithSnapshots {
     // make sure the file under bar is deleted 
     final Path barInS0 = SnapshotTestHelper.getSnapshotPath(test, "s0",
         "foo/bar");
-    INodeDirectoryWithSnapshot barNode = (INodeDirectoryWithSnapshot) fsdir
-        .getINode(barInS0.toString());
+    INodeDirectory barNode = fsdir.getINode(barInS0.toString()).asDirectory();
     assertEquals(0, barNode.getChildrenList(null).size());
     List<DirectoryDiff> diffList = barNode.getDiffs().asList();
     assertEquals(1, diffList.size());

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java

@@ -36,7 +36,7 @@ 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.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.junit.After;
 import org.junit.Before;
@@ -92,12 +92,12 @@ public class TestSetQuotaWithSnapshot {
     INodeDirectory subNode = INodeDirectory.valueOf(
         fsdir.getINode(sub.toString()), sub);
     // subNode should be a INodeDirectory, but not an INodeDirectoryWithSnapshot
-    assertFalse(subNode instanceof INodeDirectoryWithSnapshot);
+    assertFalse(subNode.isWithSnapshot());
     
     hdfs.setQuota(sub, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
     subNode = INodeDirectory.valueOf(fsdir.getINode(sub.toString()), sub);
     assertTrue(subNode.isQuotaSet());
-    assertFalse(subNode instanceof INodeDirectoryWithSnapshot);
+    assertFalse(subNode.isWithSnapshot());
   }
   
   /**
@@ -150,8 +150,8 @@ public class TestSetQuotaWithSnapshot {
     DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
     hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
     INode subNode = fsdir.getINode4Write(subDir.toString());
-    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
-    List<DirectoryDiff> diffList = ((INodeDirectoryWithSnapshot) subNode).getDiffs().asList();
+    assertTrue(subNode.asDirectory().isWithSnapshot());
+    List<DirectoryDiff> diffList = subNode.asDirectory().getDiffs().asList();
     assertEquals(1, diffList.size());
     assertEquals("s2", Snapshot.getSnapshotName(diffList.get(0).snapshot));
     List<INode> createdList = diffList.get(0).getChildrenDiff().getList(ListType.CREATED);

+ 7 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
@@ -311,9 +311,9 @@ public class TestSnapshotDeletion {
     // make sure the whole subtree of sub is stored correctly in snapshot
     Path snapshotSub = SnapshotTestHelper.getSnapshotPath(dir, "s1",
         sub.getName());
-    INodeDirectoryWithSnapshot snapshotNode4Sub = 
-        (INodeDirectoryWithSnapshot) fsdir.getINode(snapshotSub.toString());
-    assertEquals(INodeDirectoryWithSnapshot.class, snapshotNode4Sub.getClass());
+    INodeDirectory snapshotNode4Sub = fsdir.getINode(snapshotSub.toString())
+        .asDirectory();
+    assertTrue(snapshotNode4Sub.isWithSnapshot());
     // the snapshot copy of sub has only one child subsub.
     // newFile should have been destroyed
     assertEquals(1, snapshotNode4Sub.getChildrenList(null).size());
@@ -323,8 +323,7 @@ public class TestSnapshotDeletion {
     // check the snapshot copy of subsub, which is contained in the subtree of
     // sub's snapshot copy
     INode snapshotNode4Subsub = snapshotNode4Sub.getChildrenList(null).get(0);
-    assertEquals(INodeDirectoryWithSnapshot.class,
-        snapshotNode4Subsub.getClass());
+    assertTrue(snapshotNode4Subsub.asDirectory().isWithSnapshot());
     assertTrue(snapshotNode4Sub == snapshotNode4Subsub.getParent());
     // check the children of subsub
     INodeDirectory snapshotSubsubDir = (INodeDirectory) snapshotNode4Subsub;
@@ -478,8 +477,8 @@ public class TestSnapshotDeletion {
     DirectoryDiffList diffList = dirNode.getDiffs();
     assertEquals(1, diffList.asList().size());
     assertEquals("s1", diffList.getLast().snapshot.getRoot().getLocalName());
-    diffList = ((INodeDirectoryWithSnapshot) fsdir.getINode(
-        metaChangeDir.toString())).getDiffs();
+    diffList = fsdir.getINode(metaChangeDir.toString()).asDirectory()
+        .getDiffs();
     assertEquals(0, diffList.asList().size());
     
     // check 2. noChangeDir and noChangeFile are still there

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

@@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


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

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

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

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

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

@@ -71,6 +71,12 @@ Trunk (Unreleased)
     MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing.
     (Srikanth Sundarrajan via amareshwari)
 
+    MAPREDUCE-5197. Add a service for checkpointing task state.
+    (Carlo Curino via cdouglas)
+
+    MAPREDUCE-5189. Add policies and wiring to respond to preemption requests
+    from YARN. (Carlo Curino via cdouglas)
+
   BUG FIXES
 
     MAPREDUCE-4272. SortedRanges.Range#compareTo is not spec compliant.
@@ -237,6 +243,15 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5656. bzip2 codec can drop records when reading data in splits
     (jlowe)
 
+    MAPREDUCE-5623. TestJobCleanup fails because of RejectedExecutionException
+    and NPE. (jlowe)
+
+    MAPREDUCE-5679. TestJobHistoryParsing has race condition (Liyin Liang via
+    jlowe)
+
+    MAPREDUCE-5687. Fixed failure in TestYARNRunner caused by YARN-1446. (Jian He
+    via vinodkv)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -290,6 +305,9 @@ Release 2.3.0 - UNRELEASED
     event: TA_TOO_MANY_FETCH_FAILURE at KILLED for TaskAttemptImpl (Gera
     Shegalov via jlowe)
 
+    MAPREDUCE-5674. Missing start and finish time in mapred.JobStatus.
+    (Chuan Liu via cnauroth)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

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

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

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

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

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