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

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

@@ -365,6 +365,10 @@ public class LocalDirAllocator {
           totalAvailable += availableOnDisk[i];
           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
         // Keep rolling the wheel till we get a valid path
         Random r = new java.util.Random();
         Random r = new java.util.Random();
         while (numDirsSearched < numDirs && returnPath == null) {
         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>
    * </p>
    */
    */
   public static final RetryPolicy RETRY_FOREVER = new RetryForever();
   public static final RetryPolicy RETRY_FOREVER = new RetryForever();
-  
+
+  /**
+   * <p>
+   * Keep failing over forever
+   * </p>
+   */
+  public static final RetryPolicy FAILOVER_FOREVER = new FailoverForever();
+
   /**
   /**
    * <p>
    * <p>
    * Keep trying a limited number of times, waiting a fixed time between attempts,
    * Keep trying a limited number of times, waiting a fixed time between attempts,
@@ -166,6 +173,14 @@ public class RetryPolicies {
       return RetryAction.RETRY;
       return RetryAction.RETRY;
     }
     }
   }
   }
+
+  static class FailoverForever implements RetryPolicy {
+    @Override
+    public RetryAction shouldRetry(Exception e, int retries, int failovers,
+        boolean isIdempotentOrAtMostOnce) throws Exception {
+      return RetryAction.FAILOVER_AND_RETRY;
+    }
+  }
   
   
   /**
   /**
    * Retry up to maxRetries.
    * Retry up to maxRetries.

+ 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;
   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
    * The first four bytes of Hadoop RPC connections
    */
    */
   public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
   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
   // 1 : Introduce ping and server does not throw away RPCs
   // 3 : Introduce the protocol into the RPC connection header
   // 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
       @Override
       public void run() {
       public void run() {
-        LOG.info("Starting " + getName());
+        LOG.info("Starting " + Thread.currentThread().getName());
         try {
         try {
           doRunLoop();
           doRunLoop();
         } finally {
         } finally {
           try {
           try {
             readSelector.close();
             readSelector.close();
           } catch (IOException ioe) {
           } catch (IOException ioe) {
-            LOG.error("Error closing read selector in " + this.getName(), ioe);
+            LOG.error("Error closing read selector in " + Thread.currentThread().getName(), ioe);
           }
           }
         }
         }
       }
       }
@@ -589,7 +589,7 @@ public abstract class Server {
             }
             }
           } catch (InterruptedException e) {
           } catch (InterruptedException e) {
             if (running) {                      // unexpected -- log it
             if (running) {                      // unexpected -- log it
-              LOG.info(getName() + " unexpectedly interrupted", e);
+              LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
             }
             }
           } catch (IOException ex) {
           } catch (IOException ex) {
             LOG.error("Error in Reader", ex);
             LOG.error("Error in Reader", ex);
@@ -620,7 +620,7 @@ public abstract class Server {
 
 
     @Override
     @Override
     public void run() {
     public void run() {
-      LOG.info(getName() + ": starting");
+      LOG.info(Thread.currentThread().getName() + ": starting");
       SERVER.set(Server.this);
       SERVER.set(Server.this);
       connectionManager.startIdleScan();
       connectionManager.startIdleScan();
       while (running) {
       while (running) {
@@ -652,7 +652,7 @@ public abstract class Server {
           closeCurrentConnection(key, e);
           closeCurrentConnection(key, e);
         }
         }
       }
       }
-      LOG.info("Stopping " + this.getName());
+      LOG.info("Stopping " + Thread.currentThread().getName());
 
 
       synchronized (this) {
       synchronized (this) {
         try {
         try {
@@ -710,14 +710,14 @@ public abstract class Server {
       try {
       try {
         count = c.readAndProcess();
         count = c.readAndProcess();
       } catch (InterruptedException ieo) {
       } catch (InterruptedException ieo) {
-        LOG.info(getName() + ": readAndProcess caught InterruptedException", ieo);
+        LOG.info(Thread.currentThread().getName() + ": readAndProcess caught InterruptedException", ieo);
         throw ieo;
         throw ieo;
       } catch (Exception e) {
       } catch (Exception e) {
         // a WrappedRpcServerException is an exception that has been sent
         // a WrappedRpcServerException is an exception that has been sent
         // to the client, so the stacktrace is unnecessary; any other
         // to the client, so the stacktrace is unnecessary; any other
         // exceptions are unexpected internal server errors and thus the
         // exceptions are unexpected internal server errors and thus the
         // stacktrace should be logged
         // stacktrace should be logged
-        LOG.info(getName() + ": readAndProcess from client " +
+        LOG.info(Thread.currentThread().getName() + ": readAndProcess from client " +
             c.getHostAddress() + " threw exception [" + e + "]",
             c.getHostAddress() + " threw exception [" + e + "]",
             (e instanceof WrappedRpcServerException) ? null : e);
             (e instanceof WrappedRpcServerException) ? null : e);
         count = -1; //so that the (count < 0) block is executed
         count = -1; //so that the (count < 0) block is executed
@@ -740,7 +740,7 @@ public abstract class Server {
         try {
         try {
           acceptChannel.socket().close();
           acceptChannel.socket().close();
         } catch (IOException e) {
         } catch (IOException e) {
-          LOG.info(getName() + ":Exception in closing listener socket. " + e);
+          LOG.info(Thread.currentThread().getName() + ":Exception in closing listener socket. " + e);
         }
         }
       }
       }
       for (Reader r : readers) {
       for (Reader r : readers) {
@@ -773,16 +773,16 @@ public abstract class Server {
 
 
     @Override
     @Override
     public void run() {
     public void run() {
-      LOG.info(getName() + ": starting");
+      LOG.info(Thread.currentThread().getName() + ": starting");
       SERVER.set(Server.this);
       SERVER.set(Server.this);
       try {
       try {
         doRunLoop();
         doRunLoop();
       } finally {
       } finally {
-        LOG.info("Stopping " + this.getName());
+        LOG.info("Stopping " + Thread.currentThread().getName());
         try {
         try {
           writeSelector.close();
           writeSelector.close();
         } catch (IOException ioe) {
         } catch (IOException ioe) {
-          LOG.error("Couldn't close write selector in " + this.getName(), ioe);
+          LOG.error("Couldn't close write selector in " + Thread.currentThread().getName(), ioe);
         }
         }
       }
       }
     }
     }
@@ -803,7 +803,7 @@ public abstract class Server {
                   doAsyncWrite(key);
                   doAsyncWrite(key);
               }
               }
             } catch (IOException e) {
             } catch (IOException e) {
-              LOG.info(getName() + ": doAsyncWrite threw exception " + e);
+              LOG.info(Thread.currentThread().getName() + ": doAsyncWrite threw exception " + e);
             }
             }
           }
           }
           long now = Time.now();
           long now = Time.now();
@@ -918,7 +918,7 @@ public abstract class Server {
           call = responseQueue.removeFirst();
           call = responseQueue.removeFirst();
           SocketChannel channel = call.connection.channel;
           SocketChannel channel = call.connection.channel;
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug(getName() + ": responding to " + call);
+            LOG.debug(Thread.currentThread().getName() + ": responding to " + call);
           }
           }
           //
           //
           // Send as much data as we can in the non-blocking fashion
           // Send as much data as we can in the non-blocking fashion
@@ -937,7 +937,7 @@ public abstract class Server {
               done = false;            // more calls pending to be sent.
               done = false;            // more calls pending to be sent.
             }
             }
             if (LOG.isDebugEnabled()) {
             if (LOG.isDebugEnabled()) {
-              LOG.debug(getName() + ": responding to " + call
+              LOG.debug(Thread.currentThread().getName() + ": responding to " + call
                   + " Wrote " + numBytes + " bytes.");
                   + " Wrote " + numBytes + " bytes.");
             }
             }
           } else {
           } else {
@@ -965,7 +965,7 @@ public abstract class Server {
               }
               }
             }
             }
             if (LOG.isDebugEnabled()) {
             if (LOG.isDebugEnabled()) {
-              LOG.debug(getName() + ": responding to " + call
+              LOG.debug(Thread.currentThread().getName() + ": responding to " + call
                   + " Wrote partial " + numBytes + " bytes.");
                   + " Wrote partial " + numBytes + " bytes.");
             }
             }
           }
           }
@@ -973,7 +973,7 @@ public abstract class Server {
         }
         }
       } finally {
       } finally {
         if (error && call != null) {
         if (error && call != null) {
-          LOG.warn(getName()+", call " + call + ": output error");
+          LOG.warn(Thread.currentThread().getName()+", call " + call + ": output error");
           done = true;               // error. no more data for this channel.
           done = true;               // error. no more data for this channel.
           closeConnection(call.connection);
           closeConnection(call.connection);
         }
         }
@@ -1105,6 +1105,9 @@ public abstract class Server {
       this.channel = channel;
       this.channel = channel;
       this.lastContact = lastContact;
       this.lastContact = lastContact;
       this.data = null;
       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.dataLengthBuffer = ByteBuffer.allocate(4);
       this.unwrappedData = null;
       this.unwrappedData = null;
       this.unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
       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;
       Throwable cause = e;
       while (cause != null) {
       while (cause != null) {
         if (cause instanceof RetriableException) {
         if (cause instanceof RetriableException) {
@@ -1223,6 +1235,18 @@ public abstract class Server {
       return e;
       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)
     private void saslProcess(RpcSaslProto saslMessage)
         throws WrappedRpcServerException, IOException, InterruptedException {
         throws WrappedRpcServerException, IOException, InterruptedException {
       if (saslContextEstablished) {
       if (saslContextEstablished) {
@@ -1239,7 +1263,7 @@ public abstract class Server {
           // attempting user could be null
           // attempting user could be null
           AUDITLOG.warn(AUTH_FAILED_FOR + this.toString() + ":"
           AUDITLOG.warn(AUTH_FAILED_FOR + this.toString() + ":"
               + attemptingUser + " (" + e.getLocalizedMessage() + ")");
               + attemptingUser + " (" + e.getLocalizedMessage() + ")");
-          throw (IOException) getCauseForInvalidToken(e);
+          throw (IOException) getTrueCause(e);
         }
         }
         
         
         if (saslServer != null && saslServer.isComplete()) {
         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)
     private RpcSaslProto processSaslMessage(RpcSaslProto saslMessage)
-        throws IOException, InterruptedException {
+        throws SaslException, IOException, AccessControlException,
+        InterruptedException {
       RpcSaslProto saslResponse = null;
       RpcSaslProto saslResponse = null;
       final SaslState state = saslMessage.getState(); // required      
       final SaslState state = saslMessage.getState(); // required      
       switch (state) {
       switch (state) {
         case NEGOTIATE: {
         case NEGOTIATE: {
           if (sentNegotiate) {
           if (sentNegotiate) {
+            // FIXME shouldn't this be SaslException?
             throw new AccessControlException(
             throw new AccessControlException(
                 "Client already attempted negotiation");
                 "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()
     public int readAndProcess()
         throws WrappedRpcServerException, IOException, InterruptedException {
         throws WrappedRpcServerException, IOException, InterruptedException {
       while (true) {
       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;
         int count = -1;
         if (dataLengthBuffer.remaining() > 0) {
         if (dataLengthBuffer.remaining() > 0) {
           count = channelRead(channel, dataLengthBuffer);       
           count = channelRead(channel, dataLengthBuffer);       
@@ -1416,9 +1471,11 @@ public abstract class Server {
         }
         }
         
         
         if (!connectionHeaderRead) {
         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) {
           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);
           count = channelRead(channel, connectionHeaderBuf);
           if (count < 0 || connectionHeaderBuf.remaining() > 0) {
           if (count < 0 || connectionHeaderBuf.remaining() > 0) {
@@ -1451,27 +1508,30 @@ public abstract class Server {
           // this may switch us into SIMPLE
           // this may switch us into SIMPLE
           authProtocol = initializeAuthContext(connectionHeaderBuf.get(2));          
           authProtocol = initializeAuthContext(connectionHeaderBuf.get(2));          
           
           
-          dataLengthBuffer.clear();
+          dataLengthBuffer.clear(); // clear to next read rpc packet len
           connectionHeaderBuf = null;
           connectionHeaderBuf = null;
           connectionHeaderRead = true;
           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();
           dataLengthBuffer.flip();
           dataLength = dataLengthBuffer.getInt();
           dataLength = dataLengthBuffer.getInt();
           checkDataLength(dataLength);
           checkDataLength(dataLength);
+          // Set buffer for reading EXACTLY the RPC-packet length and no more.
           data = ByteBuffer.allocate(dataLength);
           data = ByteBuffer.allocate(dataLength);
         }
         }
-        
+        // Now read the RPC packet
         count = channelRead(channel, data);
         count = channelRead(channel, data);
         
         
         if (data.remaining() == 0) {
         if (data.remaining() == 0) {
-          dataLengthBuffer.clear();
+          dataLengthBuffer.clear(); // to read length of future rpc packets
           data.flip();
           data.flip();
           boolean isHeaderRead = connectionContextRead;
           boolean isHeaderRead = connectionContextRead;
           processOneRpc(data.array());
           processOneRpc(data.array());
           data = null;
           data = null;
+          // the last rpc-request we processed could have simply been the
+          // connectionContext; if so continue to read the first RPC.
           if (!isHeaderRead) {
           if (!isHeaderRead) {
             continue;
             continue;
           }
           }
@@ -1508,8 +1568,16 @@ public abstract class Server {
       return authProtocol;
       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()
     private RpcSaslProto buildSaslNegotiateResponse()
-        throws IOException, InterruptedException {
+        throws InterruptedException, SaslException, IOException {
       RpcSaslProto negotiateMessage = negotiateResponse;
       RpcSaslProto negotiateMessage = negotiateResponse;
       // accelerate token negotiation by sending initial challenge
       // accelerate token negotiation by sending initial challenge
       // in the negotiation response
       // in the negotiation response
@@ -1635,8 +1703,11 @@ public abstract class Server {
     /**
     /**
      * Process a wrapped RPC Request - unwrap the SASL packet and process
      * Process a wrapped RPC Request - unwrap the SASL packet and process
      * each embedded RPC request 
      * 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 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
      * @throws InterruptedException
      */    
      */    
     private void unwrapPacketAndProcessRpcs(byte[] inBuf)
     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
      * @param buf - contains the RPC request header and the rpc request
      * @throws IOException - internal error that should not be returned to
      * @throws IOException - internal error that should not be returned to
      *         client, typically failure to respond to client
      *         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
      *         Listener thread
      * @throws InterruptedException
      * @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 header - RPC request header
      * @param dis - stream to request payload
      * @param dis - stream to request payload
      * @throws WrappedRpcServerException - due to fatal rpc layer issues such
      * @throws WrappedRpcServerException - due to fatal rpc layer issues such
@@ -1803,7 +1885,8 @@ public abstract class Server {
      * @param dis - stream to request payload
      * @param dis - stream to request payload
      * @throws WrappedRpcServerException - setup failed due to SASL
      * @throws WrappedRpcServerException - setup failed due to SASL
      *         negotiation failure, premature or invalid connection context,
      *         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 IOException - failed to send a response back to the client
      * @throws InterruptedException
      * @throws InterruptedException
      */
      */
@@ -1928,7 +2011,7 @@ public abstract class Server {
 
 
     @Override
     @Override
     public void run() {
     public void run() {
-      LOG.debug(getName() + ": starting");
+      LOG.debug(Thread.currentThread().getName() + ": starting");
       SERVER.set(Server.this);
       SERVER.set(Server.this);
       ByteArrayOutputStream buf = 
       ByteArrayOutputStream buf = 
         new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE);
         new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE);
@@ -1936,7 +2019,7 @@ public abstract class Server {
         try {
         try {
           final Call call = callQueue.take(); // pop the queue; maybe blocked here
           final Call call = callQueue.take(); // pop the queue; maybe blocked here
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug(getName() + ": " + call + " for RpcKind " + call.rpcKind);
+            LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
           }
           }
           String errorClass = null;
           String errorClass = null;
           String error = null;
           String error = null;
@@ -1969,7 +2052,7 @@ public abstract class Server {
             if (e instanceof UndeclaredThrowableException) {
             if (e instanceof UndeclaredThrowableException) {
               e = e.getCause();
               e = e.getCause();
             }
             }
-            String logMsg = getName() + ", call " + call + ": error: " + e;
+            String logMsg = Thread.currentThread().getName() + ", call " + call + ": error: " + e;
             if (e instanceof RuntimeException || e instanceof Error) {
             if (e instanceof RuntimeException || e instanceof Error) {
               // These exception types indicate something is probably wrong
               // These exception types indicate something is probably wrong
               // on the server side, as opposed to just a normal exceptional
               // on the server side, as opposed to just a normal exceptional
@@ -2018,13 +2101,13 @@ public abstract class Server {
           }
           }
         } catch (InterruptedException e) {
         } catch (InterruptedException e) {
           if (running) {                          // unexpected -- log it
           if (running) {                          // unexpected -- log it
-            LOG.info(getName() + " unexpectedly interrupted", e);
+            LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
           }
           }
         } catch (Exception e) {
         } catch (Exception e) {
-          LOG.info(getName() + " caught an exception", e);
+          LOG.info(Thread.currentThread().getName() + " caught an exception", e);
         }
         }
       }
       }
-      LOG.debug(getName() + ": exiting");
+      LOG.debug(Thread.currentThread().getName() + ": exiting");
     }
     }
 
 
   }
   }

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

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

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

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

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

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

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

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

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

@@ -216,39 +216,37 @@ Trunk (Unreleased)
     and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
     and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
     (jing9 via szetszwo)
     (jing9 via szetszwo)
 
 
-    HDFS-5538. URLConnectionFactory should pick up the SSL related configuration 
-    by default. (Haohui Mai via jing9)
-
     HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
     HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
     with DirectoryWithQuotaFeature.  (szetszwo)
     with DirectoryWithQuotaFeature.  (szetszwo)
 
 
     HDFS-5556. Add some more NameNode cache statistics, cache pool stats
     HDFS-5556. Add some more NameNode cache statistics, cache pool stats
     (cmccabe)
     (cmccabe)
 
 
-    HDFS-5545. Allow specifying endpoints for listeners in HttpServer. (Haohui
-    Mai via jing9)
-
     HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
     HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
 
 
     HDFS-5430. Support TTL on CacheDirectives. (wang)
     HDFS-5430. Support TTL on CacheDirectives. (wang)
 
 
-    HDFS-5536. Implement HTTP policy for Namenode and DataNode. (Haohui Mai via
-    jing9)
-
     HDFS-5630. Hook up cache directive and pool usage statistics. (wang)
     HDFS-5630. Hook up cache directive and pool usage statistics. (wang)
 
 
-    HDFS-5312. Generate HTTP / HTTPS URL in DFSUtil#getInfoServer() based on the 
-    configured http policy. (Haohui Mai via jing9)
-
     HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
     HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
     FileWithSnapshotFeature.  (jing9 via szetszwo)
     FileWithSnapshotFeature.  (jing9 via szetszwo)
 
 
-    HDFS-5629. Support HTTPS in JournalNode and SecondaryNameNode. 
-    (Haohui Mai via jing9)
+    HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
+    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
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
 
+    HDFS-5665. Remove the unnecessary writeLock while initializing CacheManager
+    in FsNameSystem Ctor. (Uma Maheswara Rao G via Andrew Wang)
+
   BUG FIXES
   BUG FIXES
     HADOOP-9635 Fix potential Stack Overflow in DomainSocket.c (V. Karthik Kumar
     HADOOP-9635 Fix potential Stack Overflow in DomainSocket.c (V. Karthik Kumar
                 via cmccabe)
                 via cmccabe)
@@ -444,6 +442,12 @@ Trunk (Unreleased)
 
 
     HDFS-5626. dfsadmin -report shows incorrect cache values. (cmccabe)
     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
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
 
 
     HDFS-4985. Add storage type to the protocol and expose it in block report
     HDFS-4985. Add storage type to the protocol and expose it in block report
@@ -722,6 +726,34 @@ Release 2.4.0 - UNRELEASED
     HDFS-5637. Try to refeatchToken while local read InvalidToken occurred.
     HDFS-5637. Try to refeatchToken while local read InvalidToken occurred.
     (Liang Xie via junping_du)
     (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
   OPTIMIZATIONS
 
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
     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.
     HDFS-5580. Fix infinite loop in Balancer.waitForMoveCompletion.
     (Binglin Chang via junping_du)
     (Binglin Chang via junping_du)
 
 
+    HDFS-5676. fix inconsistent synchronization of CachingStrategy (cmccabe)
+
 Release 2.3.0 - UNRELEASED
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -804,6 +838,12 @@ Release 2.3.0 - UNRELEASED
     HDFS-4983. Numeric usernames do not work with WebHDFS FS. (Yongjun Zhang via
     HDFS-4983. Numeric usernames do not work with WebHDFS FS. (Yongjun Zhang via
     jing9)
     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
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -947,6 +987,13 @@ Release 2.3.0 - UNRELEASED
 
 
     HDFS-4201. NPE in BPServiceActor#sendHeartBeat. (jxiang via cmccabe)
     HDFS-4201. NPE in BPServiceActor#sendHeartBeat. (jxiang via cmccabe)
 
 
+    HDFS-5666. Fix inconsistent synchronization in BPOfferService (jxiang via cmccabe)
+    
+    HDFS-5657. race condition causes writeback state error in NFS gateway (brandonli)
+
+    HDFS-5661. Browsing FileSystem via web ui, should use datanode's fqdn instead of ip 
+    address. (Benoy Antony via jing9)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

+ 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;
     return false;
   }
   }
   
   
-  String getBlockPoolId() {
+  synchronized String getBlockPoolId() {
     if (bpNSInfo != null) {
     if (bpNSInfo != null) {
       return bpNSInfo.getBlockPoolID();
       return bpNSInfo.getBlockPoolID();
     } else {
     } else {
@@ -163,7 +163,7 @@ class BPOfferService {
   }
   }
 
 
   @Override
   @Override
-  public String toString() {
+  public synchronized String toString() {
     if (bpNSInfo == null) {
     if (bpNSInfo == null) {
       // If we haven't yet connected to our NN, we don't yet know our
       // If we haven't yet connected to our NN, we don't yet know our
       // own block pool ID.
       // 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.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
 import java.net.SocketTimeoutException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
@@ -273,7 +274,8 @@ class BPServiceActor implements Runnable {
   private void reportReceivedDeletedBlocks() throws IOException {
   private void reportReceivedDeletedBlocks() throws IOException {
 
 
     // Generate a list of the pending reports for each storage under the lock
     // 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) {
     synchronized (pendingIncrementalBRperStorage) {
       for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
       for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
            pendingIncrementalBRperStorage.entrySet()) {
            pendingIncrementalBRperStorage.entrySet()) {
@@ -286,33 +288,34 @@ class BPServiceActor implements Runnable {
           pendingReceivedRequests =
           pendingReceivedRequests =
               (pendingReceivedRequests > rdbi.length ?
               (pendingReceivedRequests > rdbi.length ?
                   (pendingReceivedRequests - rdbi.length) : 0);
                   (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
     // 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
             // 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
             // blocks back onto our queue, but only in the case where we
             // didn't put something newer in the meantime.
             // didn't put something newer in the meantime.
             PerStoragePendingIncrementalBR perStorageMap =
             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.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
+import java.io.EOFException;
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 
 
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
@@ -67,7 +70,29 @@ public class BlockMetadataHeader {
     return checksum;
     return checksum;
   }
   }
 
 
- 
+  /**
+   * Read the header without changing the position of the FileChannel.
+   *
+   * @param fc The FileChannel to read.
+   * @return the Metadata Header.
+   * @throws IOException on error.
+   */
+  public static BlockMetadataHeader preadHeader(FileChannel fc)
+      throws IOException {
+    byte arr[] = new byte[2 + DataChecksum.HEADER_LEN];
+    ByteBuffer buf = ByteBuffer.wrap(arr);
+
+    while (buf.hasRemaining()) {
+      if (fc.read(buf, 0) <= 0) {
+        throw new EOFException("unexpected EOF while reading " +
+            "metadata file header");
+      }
+    }
+    short version = (short)((arr[0] << 8) | (arr[1] & 0xff));
+    DataChecksum dataChecksum = DataChecksum.newDataChecksum(arr, 2);
+    return new BlockMetadataHeader(version, dataChecksum);
+  }
+
   /**
   /**
    * This reads all the fields till the beginning of checksum.
    * This reads all the fields till the beginning of checksum.
    * @param in 
    * @param in 

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

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

+ 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);
       storageInfo = new StorageInfo(nsInfo);
     }
     }
 
 
-    checkDatanodeUuid();
-
     DatanodeID dnId = new DatanodeID(
     DatanodeID dnId = new DatanodeID(
         streamingAddr.getAddress().getHostAddress(), hostName, 
         streamingAddr.getAddress().getHostAddress(), hostName, 
         storage.getDatanodeUuid(), getXferPort(), getInfoPort(),
         storage.getDatanodeUuid(), getXferPort(), getInfoPort(),
@@ -965,6 +963,9 @@ public class DataNode extends Configured
           + ";nsInfo=" + nsInfo + ";dnuuid=" + storage.getDatanodeUuid());
           + ";nsInfo=" + nsInfo + ";dnuuid=" + storage.getDatanodeUuid());
     }
     }
 
 
+    // If this is a newly formatted DataNode then assign a new DatanodeUuid.
+    checkDatanodeUuid();
+
     synchronized(this)  {
     synchronized(this)  {
       if (data == null) {
       if (data == null) {
         data = factory.newInstance(this, storage, conf);
         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.
    * Get the default chunk size.
    * @param conf the configuration
    * @param conf the configuration
@@ -228,7 +216,7 @@ public class DatanodeJspHelper {
       }
       }
     }
     }
     out.print("<br><a href=\"///"
     out.print("<br><a href=\"///"
-        + canonicalize(nnAddr) + ":"
+        + JspHelper.canonicalize(nnAddr) + ":"
         + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
         + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
     dfs.close();
     dfs.close();
   }
   }
@@ -359,7 +347,7 @@ public class DatanodeJspHelper {
     // generate a table and dump the info
     // generate a table and dump the info
     out.println("\n<table>");
     out.println("\n<table>");
     
     
-    String nnCanonicalName = canonicalize(nnAddr);
+    String nnCanonicalName = JspHelper.canonicalize(nnAddr);
     for (LocatedBlock cur : blocks) {
     for (LocatedBlock cur : blocks) {
       out.print("<tr>");
       out.print("<tr>");
       final String blockidstring = Long.toString(cur.getBlock().getBlockId());
       final String blockidstring = Long.toString(cur.getBlock().getBlockId());

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

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

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

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

+ 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}. 
  * 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_NAMESPACE_QUOTA = Long.MAX_VALUE;
   public static final long DEFAULT_DISKSPACE_QUOTA = HdfsConstants.QUOTA_RESET;
   public static final long DEFAULT_DISKSPACE_QUOTA = HdfsConstants.QUOTA_RESET;
 
 
@@ -153,6 +153,10 @@ public final class DirectoryWithQuotaFeature extends INodeDirectory.Feature {
     verifyNamespaceQuota(nsDelta);
     verifyNamespaceQuota(nsDelta);
     verifyDiskspaceQuota(dsDelta);
     verifyDiskspaceQuota(dsDelta);
   }
   }
+  
+  boolean isQuotaSet() {
+    return nsQuota >= 0 || dsQuota >= 0;
+  }
 
 
   private String namespaceString() {
   private String namespaceString() {
     return "namespace: " + (nsQuota < 0? "-": namespace + "/" + nsQuota);
     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.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 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.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;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.Root;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.Root;
 import org.apache.hadoop.hdfs.util.ByteArray;
 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 is taken on the dst tree, changes will be recorded in the latest
     // snapshot of the src tree.
     // snapshot of the src tree.
     if (isSrcInSnapshot) {
     if (isSrcInSnapshot) {
-      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot(),
-          inodeMap);
+      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot());
       srcIIP.setLastINode(srcChild);
       srcIIP.setLastINode(srcChild);
     }
     }
     
     
@@ -692,11 +690,9 @@ public class FSDirectory implements Closeable {
         }
         }
         // update modification time of dst and the parent of src
         // update modification time of dst and the parent of src
         final INode srcParent = srcIIP.getINode(-2);
         final INode srcParent = srcIIP.getINode(-2);
-        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot(),
-            inodeMap);
+        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot());
         dstParent = dstIIP.getINode(-2); // refresh dstParent
         dstParent = dstIIP.getINode(-2); // refresh dstParent
-        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot(),
-            inodeMap);
+        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot());
         // update moved leases with new filename
         // update moved leases with new filename
         getFSNamesystem().unprotectedChangeLease(src, dst);     
         getFSNamesystem().unprotectedChangeLease(src, dst);     
 
 
@@ -734,11 +730,10 @@ public class FSDirectory implements Closeable {
         }
         }
         
         
         if (isSrcInSnapshot) {
         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 {
         } else {
           // original srcChild is not in latest snapshot, we only need to add
           // original srcChild is not in latest snapshot, we only need to add
           // the srcChild back
           // 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 is taken on the dst tree, changes will be recorded in the latest
     // snapshot of the src tree.
     // snapshot of the src tree.
     if (isSrcInSnapshot) {
     if (isSrcInSnapshot) {
-      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot(),
-          inodeMap);
+      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot());
       srcIIP.setLastINode(srcChild);
       srcIIP.setLastINode(srcChild);
     }
     }
     
     
@@ -958,11 +952,9 @@ public class FSDirectory implements Closeable {
         }
         }
 
 
         final INode srcParent = srcIIP.getINode(-2);
         final INode srcParent = srcIIP.getINode(-2);
-        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot(),
-            inodeMap);
+        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot());
         dstParent = dstIIP.getINode(-2);
         dstParent = dstIIP.getINode(-2);
-        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot(),
-            inodeMap);
+        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot());
         // update moved lease with new filename
         // update moved lease with new filename
         getFSNamesystem().unprotectedChangeLease(src, dst);
         getFSNamesystem().unprotectedChangeLease(src, dst);
 
 
@@ -1019,9 +1011,9 @@ public class FSDirectory implements Closeable {
           withCount.getReferredINode().setLocalName(srcChildName);
           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 {
         } else {
           // srcParent is not an INodeDirectoryWithSnapshot, we only need to add
           // srcParent is not an INodeDirectoryWithSnapshot, we only need to add
           // the srcChild back
           // the srcChild back
@@ -1030,9 +1022,9 @@ public class FSDirectory implements Closeable {
       }
       }
       if (undoRemoveDst) {
       if (undoRemoveDst) {
         // Rename failed - restore dst
         // 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 {
         } else {
           addLastINodeNoQuotaCheck(dstIIP, removedDst);
           addLastINodeNoQuotaCheck(dstIIP, removedDst);
         }
         }
@@ -1163,8 +1155,7 @@ public class FSDirectory implements Closeable {
     if (inode == null) {
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: " + src);
       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)
   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);
       throw new FileNotFoundException("File does not exist: " + src);
     }
     }
     if (username != null) {
     if (username != null) {
-      inode = inode.setUser(username, inodesInPath.getLatestSnapshot(),
-          inodeMap);
+      inode = inode.setUser(username, inodesInPath.getLatestSnapshot());
     }
     }
     if (groupname != null) {
     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;
       if(nodeToRemove == null) continue;
       
       
       nodeToRemove.setBlocks(null);
       nodeToRemove.setBlocks(null);
-      trgParent.removeChild(nodeToRemove, trgLatestSnapshot, null);
+      trgParent.removeChild(nodeToRemove, trgLatestSnapshot);
       inodeMap.remove(nodeToRemove);
       inodeMap.remove(nodeToRemove);
       count++;
       count++;
     }
     }
@@ -1274,8 +1264,8 @@ public class FSDirectory implements Closeable {
     // update inodeMap
     // update inodeMap
     removeFromInodeMap(Arrays.asList(allSrcInodes));
     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)
     // update quota on the parent directory ('count' files removed, 0 space)
     unprotectedUpdateCount(trgIIP, trgINodes.length-1, -count, 0);
     unprotectedUpdateCount(trgIIP, trgINodes.length-1, -count, 0);
   }
   }
@@ -1419,7 +1409,7 @@ public class FSDirectory implements Closeable {
 
 
     // record modification
     // record modification
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
-    targetNode = targetNode.recordModification(latestSnapshot, inodeMap);
+    targetNode = targetNode.recordModification(latestSnapshot);
     iip.setLastINode(targetNode);
     iip.setLastINode(targetNode);
 
 
     // Remove the node from the namespace
     // Remove the node from the namespace
@@ -1430,7 +1420,7 @@ public class FSDirectory implements Closeable {
 
 
     // set the parent's modification time
     // set the parent's modification time
     final INodeDirectory parent = targetNode.getParent();
     final INodeDirectory parent = targetNode.getParent();
-    parent.updateModificationTime(mtime, latestSnapshot, inodeMap);
+    parent.updateModificationTime(mtime, latestSnapshot);
     if (removed == 0) {
     if (removed == 0) {
       return 0;
       return 0;
     }
     }
@@ -2203,8 +2193,7 @@ public class FSDirectory implements Closeable {
     final INodeDirectory parent = inodes[pos-1].asDirectory();
     final INodeDirectory parent = inodes[pos-1].asDirectory();
     boolean added = false;
     boolean added = false;
     try {
     try {
-      added = parent.addChild(child, true, iip.getLatestSnapshot(),
-          inodeMap);
+      added = parent.addChild(child, true, iip.getLatestSnapshot());
     } catch (QuotaExceededException e) {
     } catch (QuotaExceededException e) {
       updateCountNoQuotaCheck(iip, pos,
       updateCountNoQuotaCheck(iip, pos,
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
@@ -2242,7 +2231,7 @@ public class FSDirectory implements Closeable {
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
     final INode last = iip.getLastINode();
     final INode last = iip.getLastINode();
     final INodeDirectory parent = iip.getINode(-2).asDirectory();
     final INodeDirectory parent = iip.getINode(-2).asDirectory();
-    if (!parent.removeChild(last, latestSnapshot, inodeMap)) {
+    if (!parent.removeChild(last, latestSnapshot)) {
       return -1;
       return -1;
     }
     }
     INodeDirectory newParent = last.getParent();
     INodeDirectory newParent = last.getParent();
@@ -2394,7 +2383,7 @@ public class FSDirectory implements Closeable {
       }
       }
 
 
       final Snapshot latest = iip.getLatestSnapshot();
       final Snapshot latest = iip.getLatestSnapshot();
-      dirNode = dirNode.recordModification(latest, inodeMap);
+      dirNode = dirNode.recordModification(latest);
       dirNode.setQuota(nsQuota, dsQuota);
       dirNode.setQuota(nsQuota, dsQuota);
       return dirNode;
       return dirNode;
     }
     }
@@ -2462,7 +2451,7 @@ public class FSDirectory implements Closeable {
     assert hasWriteLock();
     assert hasWriteLock();
     boolean status = false;
     boolean status = false;
     if (mtime != -1) {
     if (mtime != -1) {
-      inode = inode.setModificationTime(mtime, latest, inodeMap);
+      inode = inode.setModificationTime(mtime, latest);
       status = true;
       status = true;
     }
     }
     if (atime != -1) {
     if (atime != -1) {
@@ -2473,7 +2462,7 @@ public class FSDirectory implements Closeable {
       if (atime <= inodeTime + getFSNamesystem().getAccessTimePrecision() && !force) {
       if (atime <= inodeTime + getFSNamesystem().getAccessTimePrecision() && !force) {
         status =  false;
         status =  false;
       } else {
       } else {
-        inode.setAccessTime(atime, latest, inodeMap);
+        inode.setAccessTime(atime, latest);
         status = true;
         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 long totalTimeTransactions;  // total time for all transactions
   private NameNodeMetrics metrics;
   private NameNodeMetrics metrics;
 
 
-  private NNStorage storage;
-  private Configuration conf;
+  private final NNStorage storage;
+  private final Configuration conf;
   
   
-  private List<URI> editsDirs;
+  private final List<URI> editsDirs;
 
 
   private ThreadLocal<OpInstanceCache> cache =
   private ThreadLocal<OpInstanceCache> cache =
       new ThreadLocal<OpInstanceCache>() {
       new ThreadLocal<OpInstanceCache>() {
@@ -176,7 +176,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
   /**
    * The edit directories that are shared between primary and secondary.
    * The edit directories that are shared between primary and secondary.
    */
    */
-  private List<URI> sharedEditsDirs;
+  private final List<URI> sharedEditsDirs;
 
 
   private static class TransactionId {
   private static class TransactionId {
     public long txid;
     public long txid;
@@ -203,10 +203,6 @@ public class FSEditLog implements LogsPurgeable {
    * @param editsDirs List of journals to use
    * @param editsDirs List of journals to use
    */
    */
   FSEditLog(Configuration conf, NNStorage storage, List<URI> editsDirs) {
   FSEditLog(Configuration conf, NNStorage storage, List<URI> editsDirs) {
-    init(conf, storage, editsDirs);
-  }
-  
-  private void init(Configuration conf, NNStorage storage, List<URI> editsDirs) {
     isSyncRunning = false;
     isSyncRunning = false;
     this.conf = conf;
     this.conf = conf;
     this.storage = storage;
     this.storage = storage;

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

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

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

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

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

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

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

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

+ 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.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 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.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.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.Diff;
 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);
   abstract void setUser(String user);
 
 
   /** Set user */
   /** Set user */
-  final INode setUser(String user, Snapshot latest, INodeMap inodeMap)
+  final INode setUser(String user, Snapshot latest)
       throws QuotaExceededException {
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setUser(user);
     nodeToUpdate.setUser(user);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }
@@ -120,9 +119,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setGroup(String group);
   abstract void setGroup(String group);
 
 
   /** Set group */
   /** Set group */
-  final INode setGroup(String group, Snapshot latest, INodeMap inodeMap)
+  final INode setGroup(String group, Snapshot latest)
       throws QuotaExceededException {
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setGroup(group);
     nodeToUpdate.setGroup(group);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }
@@ -145,9 +144,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setPermission(FsPermission permission);
   abstract void setPermission(FsPermission permission);
 
 
   /** Set the {@link FsPermission} of this {@link INode} */
   /** 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);
     nodeToUpdate.setPermission(permission);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }
@@ -231,14 +230,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *
    *
    * @param latest the latest snapshot that has been taken.
    * @param latest the latest snapshot that has been taken.
    *        Note that it is null if no snapshots have 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.
    * @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
    *         However, in some cases, this inode may be replaced with a new inode
    *         for maintaining snapshots. The current inode is then the 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. */
   /** Check whether it's a reference. */
   public boolean isReference() {
   public boolean isReference() {
@@ -318,7 +315,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * Call recordModification(..) to capture the current states.
    * Call recordModification(..) to capture the current states.
    * Mark the INode as deleted.
    * 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. 
    * Call recordModification(..) to capture the current states. 
    * Destroy files/directories created after the latest snapshot 
    * Destroy files/directories created after the latest snapshot 
    * (i.e., the inodes stored in the created list of 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.2 To clean {@link INodeDirectory}: recursively clean its children.
    * 2.3 To clean INodeFile with snapshot: delete the corresponding snapshot in
    * 2.3 To clean INodeFile with snapshot: delete the corresponding snapshot in
    * its diff list.
    * 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.
    * snapshot in its diff list. Recursively clean its children.
    * </pre>
    * </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. */
   /** 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. */
   /** Set the last modification time of inode. */
   public abstract void setModificationTime(long modificationTime);
   public abstract void setModificationTime(long modificationTime);
 
 
   /** Set the last modification time of inode. */
   /** Set the last modification time of inode. */
   public final INode setModificationTime(long modificationTime,
   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);
     nodeToUpdate.setModificationTime(modificationTime);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }
@@ -611,9 +608,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   /**
   /**
    * Set last access time of inode.
    * 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);
     nodeToUpdate.setAccessTime(accessTime);
     return nodeToUpdate;
     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.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 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.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;
+import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
@@ -45,20 +47,6 @@ import com.google.common.base.Preconditions;
  */
  */
 public class INodeDirectory extends INodeWithAdditionalFields
 public class INodeDirectory extends INodeWithAdditionalFields
     implements INodeDirectoryAttributes {
     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. */
   /** Cast INode to INodeDirectory. */
   public static INodeDirectory valueOf(INode inode, Object path
   public static INodeDirectory valueOf(INode inode, Object path
@@ -78,9 +66,6 @@ public class INodeDirectory extends INodeWithAdditionalFields
 
 
   private List<INode> children = null;
   private List<INode> children = null;
   
   
-  /** A linked list of {@link Feature}s. */
-  private Feature headFeature = null;
-
   /** constructor */
   /** constructor */
   public INodeDirectory(long id, byte[] name, PermissionStatus permissions,
   public INodeDirectory(long id, byte[] name, PermissionStatus permissions,
       long mtime) {
       long mtime) {
@@ -102,7 +87,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
       }
       }
     }
     }
     if (copyFeatures) {
     if (copyFeatures) {
-      this.headFeature = other.headFeature;
+      this.features = other.features;
     }
     }
   }
   }
 
 
@@ -160,7 +145,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * otherwise, return null.
    * otherwise, return null.
    */
    */
   public final DirectoryWithQuotaFeature getDirectoryWithQuotaFeature() {
   public final DirectoryWithQuotaFeature getDirectoryWithQuotaFeature() {
-    for(Feature f = headFeature; f != null; f = f.nextFeature) {
+    for (Feature f : features) {
       if (f instanceof DirectoryWithQuotaFeature) {
       if (f instanceof DirectoryWithQuotaFeature) {
         return (DirectoryWithQuotaFeature)f;
         return (DirectoryWithQuotaFeature)f;
       }
       }
@@ -182,51 +167,52 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return quota;
     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) {
   private int searchChildren(byte[] name) {
     return children == null? -1: Collections.binarySearch(children, 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}. */
   /** Replace itself with an {@link INodeDirectorySnapshottable}. */
@@ -235,16 +221,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
     Preconditions.checkState(!(this instanceof INodeDirectorySnapshottable),
     Preconditions.checkState(!(this instanceof INodeDirectorySnapshottable),
         "this is already an INodeDirectorySnapshottable, this=%s", this);
         "this is already an INodeDirectorySnapshottable, this=%s", this);
     final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(this);
     final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(this);
-    replaceSelf(s, inodeMap).saveSelf2Snapshot(latest, this);
+    replaceSelf(s, inodeMap).getDirectoryWithSnapshotFeature().getDiffs()
+        .saveSelf2Snapshot(latest, s, this);
     return s;
     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}. */
   /** Replace itself with {@link INodeDirectory}. */
   public INodeDirectory replaceSelf4INodeDirectory(final INodeMap inodeMap) {
   public INodeDirectory replaceSelf4INodeDirectory(final INodeMap inodeMap) {
     Preconditions.checkState(getClass() != INodeDirectory.class,
     Preconditions.checkState(getClass() != INodeDirectory.class,
@@ -270,7 +251,13 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return newDir;
     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,
   public void replaceChild(INode oldChild, final INode newChild,
       final INodeMap inodeMap) {
       final INodeMap inodeMap) {
     Preconditions.checkNotNull(children);
     Preconditions.checkNotNull(children);
@@ -281,24 +268,24 @@ public class INodeDirectory extends INodeWithAdditionalFields
             .asReference().getReferredINode());
             .asReference().getReferredINode());
     oldChild = children.get(i);
     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
     // update the inodeMap
     if (inodeMap != null) {
     if (inodeMap != null) {
       inodeMap.put(newChild);
       inodeMap.put(newChild);
-    }
+    }    
   }
   }
 
 
   INodeReference.WithName replaceChild4ReferenceWithName(INode oldChild,
   INodeReference.WithName replaceChild4ReferenceWithName(INode oldChild,
@@ -323,14 +310,18 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
   }
 
 
   @Override
   @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.
    * @return the child inode, which may be replaced.
    */
    */
   public INode saveChild2Snapshot(final INode child, final Snapshot latest,
   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) {
     if (latest == null) {
       return child;
       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.
    * @return the child inode.
    */
    */
   public INode getChild(byte[] name, Snapshot snapshot) {
   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. */
   /** @return the {@link INodesInPath} containing only the last inode. */
@@ -424,6 +446,41 @@ public class INodeDirectory extends INodeWithAdditionalFields
     }
     }
     return -nextPos;
     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.
    * 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
    * @param setModTime set modification time for the parent node
    *                   not needed when replaying the addition and 
    *                   not needed when replaying the addition and 
    *                   the parent already has the proper mod time
    *                   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; 
    * @return false if the child with this name already exists; 
    *         otherwise, return true;
    *         otherwise, return true;
    */
    */
   public boolean addChild(INode node, final boolean setModTime,
   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());
     final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
     if (low >= 0) {
       return false;
       return false;
     }
     }
 
 
     if (isInLatestSnapshot(latest)) {
     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);
     addChild(node, low);
     if (setModTime) {
     if (setModTime) {
       // update modification time of the parent directory
       // update modification time of the parent directory
-      updateModificationTime(node.getModificationTime(), latest, inodeMap);
+      updateModificationTime(node.getModificationTime(), latest);
     }
     }
     return true;
     return true;
   }
   }
 
 
-
-  /** The same as addChild(node, false, null, false) */
   public boolean addChild(INode node) {
   public boolean addChild(INode node) {
     final int low = searchChildren(node.getLocalNameBytes());
     final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
     if (low >= 0) {
@@ -488,21 +543,34 @@ public class INodeDirectory extends INodeWithAdditionalFields
   @Override
   @Override
   public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
   public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
       int lastSnapshotId) {
       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;
       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 {
     } else {
+      useCache = q != null && !q.isQuotaSet() ? false : useCache;
       return computeDirectoryQuotaUsage(counts, useCache, lastSnapshotId);
       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) {
     if (children != null) {
       for (INode child : children) {
       for (INode child : children) {
         child.computeQuotaUsage(counts, useCache, lastSnapshotId);
         child.computeQuotaUsage(counts, useCache, lastSnapshotId);
@@ -514,12 +582,21 @@ public class INodeDirectory extends INodeWithAdditionalFields
   /** Add quota usage for this inode excluding children. */
   /** Add quota usage for this inode excluding children. */
   public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
   public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
     counts.add(Quota.NAMESPACE, 1);
     counts.add(Quota.NAMESPACE, 1);
+    // include the diff list
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.computeQuotaUsage4CurrentDirectory(counts);
+    }
     return counts;
     return counts;
   }
   }
 
 
   @Override
   @Override
   public ContentSummaryComputationContext computeContentSummary(
   public ContentSummaryComputationContext computeContentSummary(
       ContentSummaryComputationContext summary) {
       ContentSummaryComputationContext summary) {
+    final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.computeContentSummary4Snapshot(summary.getCounts());
+    }
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
     if (q != null) {
     if (q != null) {
       return q.computeContentSummary(this, summary);
       return q.computeContentSummary(this, summary);
@@ -546,13 +623,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
       if (lastYieldCount == summary.getYieldCount()) {
       if (lastYieldCount == summary.getYieldCount()) {
         continue;
         continue;
       }
       }
-
       // The locks were released and reacquired. Check parent first.
       // The locks were released and reacquired. Check parent first.
       if (getParent() == null) {
       if (getParent() == null) {
         // Stop further counting and return whatever we have so far.
         // Stop further counting and return whatever we have so far.
         break;
         break;
       }
       }
-
       // Obtain the children list again since it may have been modified.
       // Obtain the children list again since it may have been modified.
       childrenList = getChildrenList(null);
       childrenList = getChildrenList(null);
       // Reposition in case the children list is changed. Decrement by 1
       // 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.
     // Increment the directory count for this directory.
     summary.getCounts().add(Content.DIRECTORY, 1);
     summary.getCounts().add(Content.DIRECTORY, 1);
-
     // Relinquish and reacquire locks if necessary.
     // Relinquish and reacquire locks if necessary.
     summary.yield();
     summary.yield();
-
     return summary;
     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. */
   /** 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
     // 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
     // to its latest previous snapshot. (besides, we also need to consider nodes
     // created after prior but before snapshot. this will be done in 
     // created after prior but before snapshot. this will be done in 
-    // INodeDirectoryWithSnapshot#cleanSubtree)
+    // DirectoryWithSnapshotFeature)
     Snapshot s = snapshot != null && prior != null ? prior : snapshot;
     Snapshot s = snapshot != null && prior != null ? prior : snapshot;
     for (INode child : getChildrenList(s)) {
     for (INode child : getChildrenList(s)) {
       if (snapshot != null && excludedNodes != null
       if (snapshot != null && excludedNodes != null
@@ -621,6 +749,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
   @Override
   @Override
   public void destroyAndCollectBlocks(final BlocksMapUpdateInfo collectedBlocks,
   public void destroyAndCollectBlocks(final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
       final List<INode> removedINodes) {
+    final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.clear(this, collectedBlocks, removedINodes);
+    }
     for (INode child : getChildrenList(null)) {
     for (INode child : getChildrenList(null)) {
       child.destroyAndCollectBlocks(collectedBlocks, removedINodes);
       child.destroyAndCollectBlocks(collectedBlocks, removedINodes);
     }
     }
@@ -633,6 +765,13 @@ public class INodeDirectory extends INodeWithAdditionalFields
       final BlocksMapUpdateInfo collectedBlocks,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes, final boolean countDiffChange)
       final List<INode> removedINodes, final boolean countDiffChange)
       throws QuotaExceededException {
       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) {
     if (prior == null && snapshot == null) {
       // destroy the whole subtree and collect blocks that should be deleted
       // destroy the whole subtree and collect blocks that should be deleted
       Quota.Counts counts = Quota.Counts.newInstance();
       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.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 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.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
@@ -41,23 +45,6 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class INodeFile extends INodeWithAdditionalFields
 public class INodeFile extends INodeWithAdditionalFields
     implements INodeFileAttributes, BlockCollection {
     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). */
   /** The same as valueOf(inode, path, false). */
   public static INodeFile valueOf(INode inode, String path
   public static INodeFile valueOf(INode inode, String path
@@ -120,8 +107,6 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
   private BlockInfo[] blocks;
   private BlockInfo[] blocks;
 
 
-  private Feature headFeature;
-
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
       long atime, BlockInfo[] blklist, short replication,
       long preferredBlockSize) {
       long preferredBlockSize) {
@@ -135,7 +120,7 @@ public class INodeFile extends INodeWithAdditionalFields
     super(that);
     super(that);
     this.header = that.header;
     this.header = that.header;
     this.blocks = that.blocks;
     this.blocks = that.blocks;
-    this.headFeature = that.headFeature;
+    this.features = that.features;
   }
   }
   
   
   public INodeFile(INodeFile that, FileDiffList diffs) {
   public INodeFile(INodeFile that, FileDiffList diffs) {
@@ -144,14 +129,6 @@ public class INodeFile extends INodeWithAdditionalFields
     this.addSnapshotFeature(diffs);
     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. */
   /** @return true unconditionally. */
   @Override
   @Override
   public final boolean isFile() {
   public final boolean isFile() {
@@ -171,7 +148,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * otherwise, return null.
    * otherwise, return null.
    */
    */
   public final FileUnderConstructionFeature getFileUnderConstructionFeature() {
   public final FileUnderConstructionFeature getFileUnderConstructionFeature() {
-    for (Feature f = this.headFeature; f != null; f = f.nextFeature) {
+    for (Feature f : features) {
       if (f instanceof FileUnderConstructionFeature) {
       if (f instanceof FileUnderConstructionFeature) {
         return (FileUnderConstructionFeature) f;
         return (FileUnderConstructionFeature) f;
       }
       }
@@ -273,6 +250,8 @@ public class INodeFile extends INodeWithAdditionalFields
   /* Start of Snapshot Feature */
   /* Start of Snapshot Feature */
 
 
   private FileWithSnapshotFeature addSnapshotFeature(FileDiffList diffs) {
   private FileWithSnapshotFeature addSnapshotFeature(FileDiffList diffs) {
+    Preconditions.checkState(!isWithSnapshot(), 
+        "File is already with snapshot");
     FileWithSnapshotFeature sf = new FileWithSnapshotFeature(diffs);
     FileWithSnapshotFeature sf = new FileWithSnapshotFeature(diffs);
     this.addFeature(sf);
     this.addFeature(sf);
     return sf;
     return sf;
@@ -283,7 +262,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * otherwise, return null.
    * otherwise, return null.
    */
    */
   public final FileWithSnapshotFeature getFileWithSnapshotFeature() {
   public final FileWithSnapshotFeature getFileWithSnapshotFeature() {
-    for (Feature f = headFeature; f != null; f = f.nextFeature) {
+    for (Feature f: features) {
       if (f instanceof FileWithSnapshotFeature) {
       if (f instanceof FileWithSnapshotFeature) {
         return (FileWithSnapshotFeature) f;
         return (FileWithSnapshotFeature) f;
       }
       }
@@ -306,25 +285,23 @@ public class INodeFile extends INodeWithAdditionalFields
   public INodeFileAttributes getSnapshotINode(final Snapshot snapshot) {
   public INodeFileAttributes getSnapshotINode(final Snapshot snapshot) {
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
     if (sf != null) {
     if (sf != null) {
-      return sf.getSnapshotINode(this, snapshot);
+      return sf.getDiffs().getSnapshotINode(snapshot, this);
     } else {
     } else {
       return this;
       return this;
     }
     }
   }
   }
 
 
   @Override
   @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
       // the file is in snapshot, create a snapshot feature if it does not have
       FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
       FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
       if (sf == null) {
       if (sf == null) {
         sf = addSnapshotFeature(null);
         sf = addSnapshotFeature(null);
       }
       }
       // record self in the diff list if necessary
       // 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;
     return this;
   }
   }
@@ -376,7 +353,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Set the replication factor of this file. */
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication, Snapshot latest,
   public final INodeFile setFileReplication(short replication, Snapshot latest,
       final INodeMap inodeMap) throws QuotaExceededException {
       final INodeMap inodeMap) throws QuotaExceededException {
-    final INodeFile nodeToUpdate = recordModification(latest, inodeMap);
+    final INodeFile nodeToUpdate = recordModification(latest);
     nodeToUpdate.setFileReplication(replication);
     nodeToUpdate.setFileReplication(replication);
     return nodeToUpdate;
     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) {
         "", "", new FsPermission((short) 0)), 0, 0) {
       
       
       @Override
       @Override
-      INode recordModification(Snapshot latest, INodeMap inodeMap)
-          throws QuotaExceededException {
+      INode recordModification(Snapshot latest) throws QuotaExceededException {
         return null;
         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.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 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 org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -103,9 +103,12 @@ public abstract class INodeReference extends INode {
       INode referred = wc.getReferredINode();
       INode referred = wc.getReferredINode();
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
         return referred.asFile().getDiffs().getPrior(wn.lastSnapshotId);
         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;
     return null;
@@ -231,9 +234,9 @@ public abstract class INodeReference extends INode {
   }
   }
   
   
   @Override
   @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
   @Override
@@ -252,9 +255,9 @@ public abstract class INodeReference extends INode {
   }
   }
 
 
   @Override
   @Override
-  final INode recordModification(Snapshot latest, final INodeMap inodeMap)
+  final INode recordModification(Snapshot latest)
       throws QuotaExceededException {
       throws QuotaExceededException {
-    referred.recordModification(latest, inodeMap);
+    referred.recordModification(latest);
     // reference is never replaced 
     // reference is never replaced 
     return this;
     return this;
   }
   }
@@ -547,9 +550,12 @@ public abstract class INodeReference extends INode {
       Snapshot snapshot = null;
       Snapshot snapshot = null;
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
         snapshot = referred.asFile().getDiffs().getPrior(lastSnapshotId);
         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;
       return snapshot;
     }
     }
@@ -634,10 +640,11 @@ public abstract class INodeReference extends INode {
         Snapshot snapshot = getSelfSnapshot(prior);
         Snapshot snapshot = getSelfSnapshot(prior);
         
         
         INode referred = getReferredINode().asReference().getReferredINode();
         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
           // recordModification before the rename
           INodeFile file = referred.asFile();
           INodeFile file = referred.asFile();
+          Preconditions.checkState(file.isWithSnapshot());
           // make sure we mark the file as deleted
           // make sure we mark the file as deleted
           file.getFileWithSnapshotFeature().deleteCurrentFile();
           file.getFileWithSnapshotFeature().deleteCurrentFile();
           try {
           try {
@@ -649,14 +656,14 @@ public abstract class INodeReference extends INode {
           } catch (QuotaExceededException e) {
           } catch (QuotaExceededException e) {
             LOG.error("should not exceed quota while snapshot deletion", 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
           // 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 {
           try {
-            INodeDirectoryWithSnapshot.destroyDstSubtree(sdir, snapshot, prior,
-                collectedBlocks, removedINodes);
+            DirectoryWithSnapshotFeature.destroyDstSubtree(dir, snapshot,
+                prior, collectedBlocks, removedINodes);
           } catch (QuotaExceededException e) {
           } catch (QuotaExceededException e) {
             LOG.error("should not exceed quota while snapshot deletion", e);
             LOG.error("should not exceed quota while snapshot deletion", e);
           }
           }
@@ -670,9 +677,12 @@ public abstract class INodeReference extends INode {
       Snapshot lastSnapshot = null;
       Snapshot lastSnapshot = null;
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
         lastSnapshot = referred.asFile().getDiffs().getLastSnapshot();
         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)) {
       if (lastSnapshot != null && !lastSnapshot.equals(prior)) {
         return lastSnapshot;
         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
   @Override
-  INode recordModification(Snapshot latest, final INodeMap inodeMap)
-      throws QuotaExceededException {
+  INode recordModification(Snapshot latest) throws QuotaExceededException {
     if (isInLatestSnapshot(latest)) {
     if (isInLatestSnapshot(latest)) {
       INodeDirectory parent = getParent();
       INodeDirectory parent = getParent();
-      parent.saveChild2Snapshot(this, latest, new INodeSymlink(this), inodeMap);
+      parent.saveChild2Snapshot(this, latest, new INodeSymlink(this));
     }
     }
     return 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}. */
   /** For implementing {@link LinkedElement}. */
   private LinkedElement next = null;
   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,
   private INodeWithAdditionalFields(INode parent, long id, byte[] name,
       long permission, long modificationTime, long accessTime) {
       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. */
   /** Update modification time if it is larger than the current value. */
   @Override
   @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());
     Preconditions.checkState(isDirectory());
     if (mtime <= modificationTime) {
     if (mtime <= modificationTime) {
       return this;
       return this;
     }
     }
-    return setModificationTime(mtime, latest, inodeMap);
+    return setModificationTime(mtime, latest);
   }
   }
 
 
   final void cloneModificationTime(INodeWithAdditionalFields that) {
   final void cloneModificationTime(INodeWithAdditionalFields that) {
@@ -262,4 +265,45 @@ public abstract class INodeWithAdditionalFields extends INode
   public final void setAccessTime(long accessTime) {
   public final void setAccessTime(long accessTime) {
     this.accessTime = 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.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 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.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;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -132,11 +132,11 @@ public class INodesInPath {
       final boolean isRef = curNode.isReference();
       final boolean isRef = curNode.isReference();
       final boolean isDir = curNode.isDirectory();
       final boolean isDir = curNode.isDirectory();
       final INodeDirectory dir = isDir? curNode.asDirectory(): null;  
       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 the path is a non-snapshot path, update the latest snapshot.
         if (!existing.isSnapshot()) {
         if (!existing.isSnapshot()) {
-          existing.updateLatestSnapshot(
-              ((INodeDirectoryWithSnapshot)dir).getLastSnapshot());
+          existing.updateLatestSnapshot(dir.getDirectoryWithSnapshotFeature()
+              .getLastSnapshot());
         }
         }
       } else if (isRef && isDir && !lastComp) {
       } else if (isRef && isDir && !lastComp) {
         // If the curNode is a reference node, need to check its dstSnapshot:
         // 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
           if (latest == null ||  // no snapshot in dst tree of rename
               dstSnapshotId >= latest.getId()) { // the above scenario 
               dstSnapshotId >= latest.getId()) { // the above scenario 
             Snapshot lastSnapshot = null;
             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);
             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.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Set;
 import java.util.Set;
@@ -36,6 +37,7 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
@@ -1006,6 +1008,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
   public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
       StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
       StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
     verifyRequest(nodeReg);
     verifyRequest(nodeReg);
+    metrics.incrBlockReceivedAndDeletedOps();
     if(blockStateChangeLog.isDebugEnabled()) {
     if(blockStateChangeLog.isDebugEnabled()) {
       blockStateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: "
       blockStateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: "
           +"from "+nodeReg+" "+receivedAndDeletedBlocks.length
           +"from "+nodeReg+" "+receivedAndDeletedBlocks.length
@@ -1238,14 +1241,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
   @Override
   @Override
   public long addCacheDirective(
   public long addCacheDirective(
-      CacheDirectiveInfo path) throws IOException {
-    return namesystem.addCacheDirective(path);
+      CacheDirectiveInfo path, EnumSet<CacheFlag> flags) throws IOException {
+    return namesystem.addCacheDirective(path, flags);
   }
   }
 
 
   @Override
   @Override
   public void modifyCacheDirective(
   public void modifyCacheDirective(
-      CacheDirectiveInfo directive) throws IOException {
-    namesystem.modifyCacheDirective(directive);
+      CacheDirectiveInfo directive, EnumSet<CacheFlag> flags) throws IOException {
+    namesystem.modifyCacheDirective(directive, flags);
   }
   }
 
 
   @Override
   @Override

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

+ 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. */
   /** 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");
     Preconditions.checkState(snapshotINode == null, "Expected snapshotINode to be null");
     snapshotINode = snapshotCopy;
     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.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 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.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 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
    *         Note that the current inode is returned if there is no change
    *         between the given snapshot and the current state. 
    *         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 D diff = getDiff(snapshot);
     final A inode = diff == null? null: diff.getSnapshotINode();
     final A inode = diff == null? null: diff.getSnapshotINode();
     return inode == null? currentINode: inode;
     return inode == null? currentINode: inode;
@@ -306,7 +306,7 @@ abstract class AbstractINodeDiffList<N extends INode,
         if (snapshotCopy == null) {
         if (snapshotCopy == null) {
           snapshotCopy = createSnapshotCopy(currentINode);
           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.List;
 import java.util.Map;
 import java.util.Map;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 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.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 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.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 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;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 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;
 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
    * The difference between the current state and a previous snapshot
    * of the children list of an INodeDirectory.
    * of the children list of an INodeDirectory.
    */
    */
   static class ChildrenDiff extends Diff<byte[], INode> {
   static class ChildrenDiff extends Diff<byte[], INode> {
     ChildrenDiff() {}
     ChildrenDiff() {}
-    
+
     private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
     private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
       super(created, deleted);
       super(created, deleted);
     }
     }
@@ -73,7 +73,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
      */
      */
     private final boolean replace(final ListType type,
     private final boolean replace(final ListType type,
         final INode oldChild, final INode newChild) {
         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());
       final int i = search(list, oldChild.getLocalNameBytes());
       if (i < 0 || list.get(i).getId() != oldChild.getId()) {
       if (i < 0 || list.get(i).getId() != oldChild.getId()) {
         return false;
         return false;
@@ -93,10 +93,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       }
       }
       return false;
       return false;
     }
     }
-    
+
     /** clear the created list */
     /** clear the created list */
-    private Quota.Counts destroyCreatedList(
-        final INodeDirectoryWithSnapshot currentINode,
+    private Quota.Counts destroyCreatedList(final INodeDirectory currentINode,
         final BlocksMapUpdateInfo collectedBlocks,
         final BlocksMapUpdateInfo collectedBlocks,
         final List<INode> removedINodes) {
         final List<INode> removedINodes) {
       Quota.Counts counts = Quota.Counts.newInstance();
       Quota.Counts counts = Quota.Counts.newInstance();
@@ -110,7 +109,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       createdList.clear();
       createdList.clear();
       return counts;
       return counts;
     }
     }
-    
+
     /** clear the deleted list */
     /** clear the deleted list */
     private Quota.Counts destroyDeletedList(
     private Quota.Counts destroyDeletedList(
         final BlocksMapUpdateInfo collectedBlocks,
         final BlocksMapUpdateInfo collectedBlocks,
@@ -124,19 +123,19 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       deletedList.clear();
       deletedList.clear();
       return counts;
       return counts;
     }
     }
-    
+
     /** Serialize {@link #created} */
     /** Serialize {@link #created} */
     private void writeCreated(DataOutput out) throws IOException {
     private void writeCreated(DataOutput out) throws IOException {
       final List<INode> created = getList(ListType.CREATED);
       final List<INode> created = getList(ListType.CREATED);
       out.writeInt(created.size());
       out.writeInt(created.size());
       for (INode node : created) {
       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();
         byte[] name = node.getLocalNameBytes();
         out.writeShort(name.length);
         out.writeShort(name.length);
         out.write(name);
         out.write(name);
       }
       }
     }
     }
-    
+
     /** Serialize {@link #deleted} */
     /** Serialize {@link #deleted} */
     private void writeDeleted(DataOutput out,
     private void writeDeleted(DataOutput out,
         ReferenceMap referenceMap) throws IOException {
         ReferenceMap referenceMap) throws IOException {
@@ -146,12 +145,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         FSImageSerialization.saveINode2Image(node, out, true, referenceMap);
         FSImageSerialization.saveINode2Image(node, out, true, referenceMap);
       }
       }
     }
     }
-    
+
     /** Serialize to out */
     /** Serialize to out */
     private void write(DataOutput out, ReferenceMap referenceMap
     private void write(DataOutput out, ReferenceMap referenceMap
         ) throws IOException {
         ) throws IOException {
       writeCreated(out);
       writeCreated(out);
-      writeDeleted(out, referenceMap);    
+      writeDeleted(out, referenceMap);
     }
     }
 
 
     /** Get the list of INodeDirectory contained in the deleted list */
     /** 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}.
      * Interpret the diff and generate a list of {@link DiffReportEntry}.
      * @param parentPath The relative path of the parent.
      * @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}
      *                    False indicates {@code diff=earlier-later}
      * @return A list of {@link DiffReportEntry} as the diff report.
      * @return A list of {@link DiffReportEntry} as the diff report.
      */
      */
     public List<DiffReportEntry> generateReport(byte[][] parentPath,
     public List<DiffReportEntry> generateReport(byte[][] parentPath,
-        INodeDirectoryWithSnapshot parent, boolean fromEarlier) {
+        boolean fromEarlier) {
       List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
       List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
       List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
       List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
       int c = 0, d = 0;
       int c = 0, d = 0;
@@ -217,7 +215,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       return dList;
       return dList;
     }
     }
   }
   }
-  
+
   /**
   /**
    * The difference of an {@link INodeDirectory} between two snapshots.
    * The difference of an {@link INodeDirectory} between two snapshots.
    */
    */
@@ -243,16 +241,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       this.childrenSize = childrenSize;
       this.childrenSize = childrenSize;
       this.diff = new ChildrenDiff(createdList, deletedList);
       this.diff = new ChildrenDiff(createdList, deletedList);
     }
     }
-    
+
     ChildrenDiff getChildrenDiff() {
     ChildrenDiff getChildrenDiff() {
       return diff;
       return diff;
     }
     }
-    
+
     /** Is the inode the root of the snapshot? */
     /** Is the inode the root of the snapshot? */
     boolean isSnapshotRoot() {
     boolean isSnapshotRoot() {
       return snapshotINode == snapshot.getRoot();
       return snapshotINode == snapshot.getRoot();
     }
     }
-    
+
     @Override
     @Override
     Quota.Counts combinePosteriorAndCollectBlocks(
     Quota.Counts combinePosteriorAndCollectBlocks(
         final INodeDirectory currentDir, final DirectoryDiff posterior,
         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
      *         Since the snapshot is read-only, the logical view of the list is
      *         never changed although the internal data structure may mutate.
      *         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>() {
       return new ReadOnlyList<INode>() {
         private List<INode> children = null;
         private List<INode> children = null;
 
 
         private List<INode> initChildren() {
         private List<INode> initChildren() {
           if (children == null) {
           if (children == null) {
             final ChildrenDiff combined = new ChildrenDiff();
             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);
               combined.combinePosterior(d.diff, null);
             }
             }
             children = combined.apply2Current(ReadOnlyList.Util.asList(
             children = combined.apply2Current(ReadOnlyList.Util.asList(
@@ -297,17 +296,17 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         public Iterator<INode> iterator() {
         public Iterator<INode> iterator() {
           return initChildren().iterator();
           return initChildren().iterator();
         }
         }
-    
+
         @Override
         @Override
         public boolean isEmpty() {
         public boolean isEmpty() {
           return childrenSize == 0;
           return childrenSize == 0;
         }
         }
-    
+
         @Override
         @Override
         public int size() {
         public int size() {
           return childrenSize;
           return childrenSize;
         }
         }
-    
+
         @Override
         @Override
         public INode get(int i) {
         public INode get(int i) {
           return initChildren().get(i);
           return initChildren().get(i);
@@ -322,9 +321,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         final Container<INode> returned = d.diff.accessPrevious(name);
         final Container<INode> returned = d.diff.accessPrevious(name);
         if (returned != null) {
         if (returned != null) {
           // the diff is able to determine the inode
           // the diff is able to determine the inode
-          return returned.getElement(); 
+          return returned.getElement();
         } else if (!checkPosterior) {
         } 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;
           return null;
         } else if (d.getPosterior() == null) {
         } else if (d.getPosterior() == null) {
           // no more posterior diff, get from current inode.
           // no more posterior diff, get from current inode.
@@ -332,12 +331,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
         }
       }
       }
     }
     }
-    
+
     @Override
     @Override
     public String toString() {
     public String toString() {
       return super.toString() + " childrenSize=" + childrenSize + ", " + diff;
       return super.toString() + " childrenSize=" + childrenSize + ", " + diff;
     }
     }
-    
+
     @Override
     @Override
     void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
     void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
       writeSnapshot(out);
       writeSnapshot(out);
@@ -386,7 +385,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
     }
     }
 
 
     /** Replace the given child in the created/deleted list, if there is any. */
     /** 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 INode newChild) {
       final List<DirectoryDiff> diffList = asList();
       final List<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
       for(int i = diffList.size() - 1; i >= 0; i--) {
@@ -397,9 +396,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       }
       }
       return false;
       return false;
     }
     }
-    
+
     /** Remove the given child in the created/deleted list, if there is any. */
     /** 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();
       final List<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
       for(int i = diffList.size() - 1; i >= 0; i--) {
         final ChildrenDiff diff = diffList.get(i).diff;
         final ChildrenDiff diff = diffList.get(i).diff;
@@ -410,84 +409,134 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       return false;
       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. */
   /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
   private final DirectoryDiffList diffs;
   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. */
   /** @return the last snapshot. */
@@ -499,204 +548,203 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
   public DirectoryDiffList getDiffs() {
   public DirectoryDiffList getDiffs() {
     return diffs;
     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);
       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
     // 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
     // false, the directory is not in the latest snapshot, thus we do not need
     // to record the removed child in any snapshot.
     // to record the removed child in any snapshot.
     // For a directory that was moved/renamed, note that if the directory is in
     // 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
     // directory while rename, and isInLatestSnapshot will return true in that
     // scenario (if all previous snapshots have been deleted, isInLatestSnapshot
     // 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.
     // 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;
     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);
     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 BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes, final boolean countDiffChange)
       final List<INode> removedINodes, final boolean countDiffChange)
       throws QuotaExceededException {
       throws QuotaExceededException {
@@ -704,12 +752,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
     Map<INode, INode> priorCreated = null;
     Map<INode, INode> priorCreated = null;
     Map<INode, INode> priorDeleted = null;
     Map<INode, INode> priorDeleted = null;
     if (snapshot == null) { // delete the current directory
     if (snapshot == null) { // delete the current directory
-      recordModification(prior, null);
+      currentINode.recordModification(prior);
       // delete everything in created list
       // delete everything in created list
       DirectoryDiff lastDiff = diffs.getLast();
       DirectoryDiff lastDiff = diffs.getLast();
       if (lastDiff != null) {
       if (lastDiff != null) {
-        counts.add(lastDiff.diff.destroyCreatedList(this, collectedBlocks,
-            removedINodes));
+        counts.add(lastDiff.diff.destroyCreatedList(currentINode,
+            collectedBlocks, removedINodes));
       }
       }
     } else {
     } else {
       // update prior
       // 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));
           collectedBlocks, removedINodes, countDiffChange));
       
       
       // check priorDiff again since it may be created during the diff deletion
       // 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));
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
     }
     }
     return counts;
     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;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 
 /**
 /**
  * Feature for file with snapshot-related information.
  * Feature for file with snapshot-related information.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class FileWithSnapshotFeature extends INodeFile.Feature {
+public class FileWithSnapshotFeature implements INode.Feature {
   private final FileDiffList diffs;
   private final FileDiffList diffs;
   private boolean isCurrentFileDeleted = false;
   private boolean isCurrentFileDeleted = false;
   
   
@@ -57,10 +56,6 @@ public class FileWithSnapshotFeature extends INodeFile.Feature {
     isCurrentFileDeleted = true;
     isCurrentFileDeleted = true;
   }
   }
 
 
-  public INodeFileAttributes getSnapshotINode(INodeFile f, Snapshot snapshot) {
-    return diffs.getSnapshotINode(snapshot, f);
-  }
-
   public FileDiffList getDiffs() {
   public FileDiffList getDiffs() {
     return diffs;
     return diffs;
   }
   }
@@ -90,7 +85,7 @@ public class FileWithSnapshotFeature extends INodeFile.Feature {
     if (snapshot == null) {
     if (snapshot == null) {
       // delete the current file while the file has snapshot feature
       // delete the current file while the file has snapshot feature
       if (!isCurrentFileDeleted()) {
       if (!isCurrentFileDeleted()) {
-        file.recordModification(prior, null);
+        file.recordModification(prior);
         deleteCurrentFile();
         deleteCurrentFile();
       }
       }
       collectBlocksAndClear(file, collectedBlocks, removedINodes);
       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.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 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.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
@@ -58,7 +60,7 @@ import com.google.common.primitives.SignedBytes;
  * by the namesystem and FSDirectory locks.
  * by the namesystem and FSDirectory locks.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
+public class INodeDirectorySnapshottable extends INodeDirectory {
   /** Limit the number of snapshot per snapshottable directory. */
   /** Limit the number of snapshot per snapshottable directory. */
   static final int SNAPSHOT_LIMIT = 1 << 16;
   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}
      * the two snapshots, while its associated value is a {@link ChildrenDiff}
      * storing the changes (creation/deletion) happened to the children (files).
      * 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,
     SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot, Snapshot start,
         Snapshot end) {
         Snapshot end) {
@@ -126,8 +128,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     }
     }
     
     
     /** Add a dir-diff pair */
     /** 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);
       dirDiffMap.put(dir, diff);
       diffMap.put(dir, relativePath);
       diffMap.put(dir, relativePath);
     }
     }
@@ -154,8 +156,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
         if (node.isDirectory()) {
         if (node.isDirectory()) {
           ChildrenDiff dirDiff = dirDiffMap.get(node);
           ChildrenDiff dirDiff = dirDiffMap.get(node);
           List<DiffReportEntry> subList = dirDiff.generateReport(
           List<DiffReportEntry> subList = dirDiff.generateReport(
-              diffMap.get(node), (INodeDirectoryWithSnapshot) node,
-              isFromEarlier());
+              diffMap.get(node), isFromEarlier());
           diffReportList.addAll(subList);
           diffReportList.addAll(subList);
         }
         }
       }
       }
@@ -183,8 +184,11 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   private int snapshotQuota = SNAPSHOT_LIMIT;
   private int snapshotQuota = SNAPSHOT_LIMIT;
 
 
   public INodeDirectorySnapshottable(INodeDirectory dir) {
   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. */
   /** @return the number of existing snapshots. */
@@ -298,8 +302,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     snapshotsByNames.add(-i - 1, s);
     snapshotsByNames.add(-i - 1, s);
 
 
     //set modification time
     //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;
     return s;
   }
   }
   
   
@@ -413,12 +417,12 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
     byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
     if (node.isDirectory()) {
     if (node.isDirectory()) {
       INodeDirectory dir = node.asDirectory();
       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) {
         if (change) {
-          diffReport.addDirDiff(sdir, relativePath, diff);
+          diffReport.addDirDiff(dir, relativePath, diff);
         }
         }
       }
       }
       ReadOnlyList<INode> children = dir.getChildrenList(diffReport
       ReadOnlyList<INode> children = dir.getChildrenList(diffReport
@@ -453,13 +457,15 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   INodeDirectory replaceSelf(final Snapshot latest, final INodeMap inodeMap)
   INodeDirectory replaceSelf(final Snapshot latest, final INodeMap inodeMap)
       throws QuotaExceededException {
       throws QuotaExceededException {
     if (latest == null) {
     if (latest == null) {
-      Preconditions.checkState(getLastSnapshot() == null,
+      Preconditions.checkState(
+          getDirectoryWithSnapshotFeature().getLastSnapshot() == null,
           "latest == null but getLastSnapshot() != null, this=%s", this);
           "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
   @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()) {
     for(; inode != null; inode = inode.getParent()) {
       if (inode.isDirectory()) {
       if (inode.isDirectory()) {
         final INodeDirectory dir = inode.asDirectory();
         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.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 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.tools.snapshot.SnapshotDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -91,8 +91,7 @@ public class SnapshotFSImageFormat {
   public static void saveDirectoryDiffList(final INodeDirectory dir,
   public static void saveDirectoryDiffList(final INodeDirectory dir,
       final DataOutput out, final ReferenceMap referenceMap
       final DataOutput out, final ReferenceMap referenceMap
       ) throws IOException {
       ) throws IOException {
-    saveINodeDiffs(dir instanceof INodeDirectoryWithSnapshot?
-        ((INodeDirectoryWithSnapshot)dir).getDiffs(): null, out, referenceMap);
+    saveINodeDiffs(dir.getDiffs(), out, referenceMap);
   }
   }
   
   
   public static void saveFileDiffList(final INodeFile file,
   public static void saveFileDiffList(final INodeFile file,
@@ -139,7 +138,7 @@ public class SnapshotFSImageFormat {
    * @return The created node.
    * @return The created node.
    */
    */
   private static INode loadCreated(byte[] createdNodeName,
   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
     // the INode in the created list should be a reference to another INode
     // in posterior SnapshotDiffs or one of the current children
     // in posterior SnapshotDiffs or one of the current children
     for (DirectoryDiff postDiff : parent.getDiffs()) {
     for (DirectoryDiff postDiff : parent.getDiffs()) {
@@ -165,7 +164,7 @@ public class SnapshotFSImageFormat {
    * @param in The {@link DataInput} to read.
    * @param in The {@link DataInput} to read.
    * @return The created list.
    * @return The created list.
    */
    */
-  private static List<INode> loadCreatedList(INodeDirectoryWithSnapshot parent,
+  private static List<INode> loadCreatedList(INodeDirectory parent,
       DataInput in) throws IOException {
       DataInput in) throws IOException {
     // read the size of the created list
     // read the size of the created list
     int createdSize = in.readInt();
     int createdSize = in.readInt();
@@ -188,7 +187,7 @@ public class SnapshotFSImageFormat {
    * @param loader The {@link Loader} instance.
    * @param loader The {@link Loader} instance.
    * @return The deleted list.
    * @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)
       List<INode> createdList, DataInput in, FSImageFormat.Loader loader)
       throws IOException {
       throws IOException {
     int deletedSize = in.readInt();
     int deletedSize = in.readInt();
@@ -239,11 +238,10 @@ public class SnapshotFSImageFormat {
   public static void loadDirectoryDiffList(INodeDirectory dir,
   public static void loadDirectoryDiffList(INodeDirectory dir,
       DataInput in, FSImageFormat.Loader loader) throws IOException {
       DataInput in, FSImageFormat.Loader loader) throws IOException {
     final int size = in.readInt();
     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++) {
       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.
    *               using.
    * @return A {@link DirectoryDiff}.
    * @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
     // 1. Read the full path of the Snapshot root to identify the Snapshot
     final Snapshot snapshot = loader.getSnapshot(in);
     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;
 package org.apache.hadoop.hdfs.tools;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 
 
@@ -25,6 +26,7 @@ import org.apache.commons.lang.WordUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
@@ -135,6 +137,7 @@ public class CacheAdmin extends Configured implements Tool {
     public String getShortUsage() {
     public String getShortUsage() {
       return "[" + getName() +
       return "[" + getName() +
           " -path <path> -pool <pool-name> " +
           " -path <path> -pool <pool-name> " +
+          "[-force] " +
           "[-replication <replication>] [-ttl <time-to-live>]]\n";
           "[-replication <replication>] [-ttl <time-to-live>]]\n";
     }
     }
 
 
@@ -146,6 +149,8 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<pool-name>", "The pool to which the directive will be " +
       listing.addRow("<pool-name>", "The pool to which the directive will be " +
           "added. You must have write permission on the cache pool "
           "added. You must have write permission on the cache pool "
           + "in order to add new directives.");
           + "in order to add new directives.");
+      listing.addRow("-force",
+          "Skips checking of cache pool resource limits.");
       listing.addRow("<replication>", "The cache replication factor to use. " +
       listing.addRow("<replication>", "The cache replication factor to use. " +
           "Defaults to 1.");
           "Defaults to 1.");
       listing.addRow("<time-to-live>", "How long the directive is " +
       listing.addRow("<time-to-live>", "How long the directive is " +
@@ -174,7 +179,7 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
         return 1;
       }
       }
       builder.setPool(poolName);
       builder.setPool(poolName);
-
+      boolean force = StringUtils.popOption("-force", args);
       String replicationString =
       String replicationString =
           StringUtils.popOptionWithArgument("-replication", args);
           StringUtils.popOptionWithArgument("-replication", args);
       if (replicationString != null) {
       if (replicationString != null) {
@@ -201,8 +206,12 @@ public class CacheAdmin extends Configured implements Tool {
         
         
       DistributedFileSystem dfs = getDFS(conf);
       DistributedFileSystem dfs = getDFS(conf);
       CacheDirectiveInfo directive = builder.build();
       CacheDirectiveInfo directive = builder.build();
+      EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
+      if (force) {
+        flags.add(CacheFlag.FORCE);
+      }
       try {
       try {
-        long id = dfs.addCacheDirective(directive);
+        long id = dfs.addCacheDirective(directive, flags);
         System.out.println("Added cache directive " + id);
         System.out.println("Added cache directive " + id);
       } catch (IOException e) {
       } catch (IOException e) {
         System.err.println(prettifyException(e));
         System.err.println(prettifyException(e));
@@ -282,7 +291,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     @Override
     public String getShortUsage() {
     public String getShortUsage() {
       return "[" + getName() +
       return "[" + getName() +
-          " -id <id> [-path <path>] [-replication <replication>] " +
+          " -id <id> [-path <path>] [-force] [-replication <replication>] " +
           "[-pool <pool-name>] [-ttl <time-to-live>]]\n";
           "[-pool <pool-name>] [-ttl <time-to-live>]]\n";
     }
     }
 
 
@@ -292,6 +301,8 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<id>", "The ID of the directive to modify (required)");
       listing.addRow("<id>", "The ID of the directive to modify (required)");
       listing.addRow("<path>", "A path to cache. The path can be " +
       listing.addRow("<path>", "A path to cache. The path can be " +
           "a directory or a file. (optional)");
           "a directory or a file. (optional)");
+      listing.addRow("-force",
+          "Skips checking of cache pool resource limits.");
       listing.addRow("<replication>", "The cache replication factor to use. " +
       listing.addRow("<replication>", "The cache replication factor to use. " +
           "(optional)");
           "(optional)");
       listing.addRow("<pool-name>", "The pool to which the directive will be " +
       listing.addRow("<pool-name>", "The pool to which the directive will be " +
@@ -322,6 +333,7 @@ public class CacheAdmin extends Configured implements Tool {
         builder.setPath(new Path(path));
         builder.setPath(new Path(path));
         modified = true;
         modified = true;
       }
       }
+      boolean force = StringUtils.popOption("-force", args);
       String replicationString =
       String replicationString =
         StringUtils.popOptionWithArgument("-replication", args);
         StringUtils.popOptionWithArgument("-replication", args);
       if (replicationString != null) {
       if (replicationString != null) {
@@ -357,8 +369,12 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
         return 1;
       }
       }
       DistributedFileSystem dfs = getDFS(conf);
       DistributedFileSystem dfs = getDFS(conf);
+      EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
+      if (force) {
+        flags.add(CacheFlag.FORCE);
+      }
       try {
       try {
-        dfs.modifyCacheDirective(builder.build());
+        dfs.modifyCacheDirective(builder.build(), flags);
         System.out.println("Modified cache directive " + idString);
         System.out.println("Modified cache directive " + idString);
       } catch (IOException e) {
       } catch (IOException e) {
         System.err.println(prettifyException(e));
         System.err.println(prettifyException(e));
@@ -536,7 +552,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     @Override
     public String getShortUsage() {
     public String getShortUsage() {
       return "[" + NAME + " <name> [-owner <owner>] " +
       return "[" + NAME + " <name> [-owner <owner>] " +
-          "[-group <group>] [-mode <mode>] [-weight <weight>]]\n";
+          "[-group <group>] [-mode <mode>] [-limit <limit>]]\n";
     }
     }
 
 
     @Override
     @Override
@@ -551,11 +567,10 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<mode>", "UNIX-style permissions for the pool. " +
       listing.addRow("<mode>", "UNIX-style permissions for the pool. " +
           "Permissions are specified in octal, e.g. 0755. " +
           "Permissions are specified in octal, e.g. 0755. " +
           "By default, this is set to " + String.format("0%03o",
           "By default, this is set to " + String.format("0%03o",
-          FsPermission.getCachePoolDefault().toShort()));
-      listing.addRow("<weight>", "Weight of the pool. " +
-          "This is a relative measure of the importance of the pool used " +
-          "during cache resource management. By default, it is set to " +
-          CachePool.DEFAULT_WEIGHT);
+          FsPermission.getCachePoolDefault().toShort()) + ".");
+      listing.addRow("<limit>", "The maximum number of bytes that can be " +
+          "cached by directives in this pool, in aggregate. By default, " +
+          "no limit is set.");
 
 
       return getShortUsage() + "\n" +
       return getShortUsage() + "\n" +
           "Add a new cache pool.\n\n" + 
           "Add a new cache pool.\n\n" + 
@@ -564,33 +579,31 @@ public class CacheAdmin extends Configured implements Tool {
 
 
     @Override
     @Override
     public int run(Configuration conf, List<String> args) throws IOException {
     public int run(Configuration conf, List<String> args) throws IOException {
+      String name = StringUtils.popFirstNonOption(args);
+      if (name == null) {
+        System.err.println("You must specify a name when creating a " +
+            "cache pool.");
+        return 1;
+      }
+      CachePoolInfo info = new CachePoolInfo(name);
+
       String owner = StringUtils.popOptionWithArgument("-owner", args);
       String owner = StringUtils.popOptionWithArgument("-owner", args);
-      if (owner == null) {
-        owner = UserGroupInformation.getCurrentUser().getShortUserName();
+      if (owner != null) {
+        info.setOwnerName(owner);
       }
       }
       String group = StringUtils.popOptionWithArgument("-group", args);
       String group = StringUtils.popOptionWithArgument("-group", args);
-      if (group == null) {
-        group = UserGroupInformation.getCurrentUser().getGroupNames()[0];
+      if (group != null) {
+        info.setGroupName(group);
       }
       }
       String modeString = StringUtils.popOptionWithArgument("-mode", args);
       String modeString = StringUtils.popOptionWithArgument("-mode", args);
-      int mode;
-      if (modeString == null) {
-        mode = FsPermission.getCachePoolDefault().toShort();
-      } else {
-        mode = Integer.parseInt(modeString, 8);
-      }
-      String weightString = StringUtils.popOptionWithArgument("-weight", args);
-      int weight;
-      if (weightString == null) {
-        weight = CachePool.DEFAULT_WEIGHT;
-      } else {
-        weight = Integer.parseInt(weightString);
+      if (modeString != null) {
+        short mode = Short.parseShort(modeString, 8);
+        info.setMode(new FsPermission(mode));
       }
       }
-      String name = StringUtils.popFirstNonOption(args);
-      if (name == null) {
-        System.err.println("You must specify a name when creating a " +
-            "cache pool.");
-        return 1;
+      String limitString = StringUtils.popOptionWithArgument("-limit", args);
+      if (limitString != null) {
+        long limit = Long.parseLong(limitString);
+        info.setLimit(limit);
       }
       }
       if (!args.isEmpty()) {
       if (!args.isEmpty()) {
         System.err.print("Can't understand arguments: " +
         System.err.print("Can't understand arguments: " +
@@ -599,11 +612,6 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
         return 1;
       }
       }
       DistributedFileSystem dfs = getDFS(conf);
       DistributedFileSystem dfs = getDFS(conf);
-      CachePoolInfo info = new CachePoolInfo(name).
-          setOwnerName(owner).
-          setGroupName(group).
-          setMode(new FsPermission((short)mode)).
-          setWeight(weight);
       try {
       try {
         dfs.addCachePool(info);
         dfs.addCachePool(info);
       } catch (IOException e) {
       } catch (IOException e) {
@@ -624,7 +632,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     @Override
     public String getShortUsage() {
     public String getShortUsage() {
       return "[" + getName() + " <name> [-owner <owner>] " +
       return "[" + getName() + " <name> [-owner <owner>] " +
-          "[-group <group>] [-mode <mode>] [-weight <weight>]]\n";
+          "[-group <group>] [-mode <mode>] [-limit <limit>]]\n";
     }
     }
 
 
     @Override
     @Override
@@ -635,11 +643,12 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<owner>", "Username of the owner of the pool");
       listing.addRow("<owner>", "Username of the owner of the pool");
       listing.addRow("<group>", "Groupname of the group of the pool.");
       listing.addRow("<group>", "Groupname of the group of the pool.");
       listing.addRow("<mode>", "Unix-style permissions of the pool in octal.");
       listing.addRow("<mode>", "Unix-style permissions of the pool in octal.");
-      listing.addRow("<weight>", "Weight of the pool.");
+      listing.addRow("<limit>", "Maximum number of bytes that can be cached " +
+          "by this pool.");
 
 
       return getShortUsage() + "\n" +
       return getShortUsage() + "\n" +
           WordUtils.wrap("Modifies the metadata of an existing cache pool. " +
           WordUtils.wrap("Modifies the metadata of an existing cache pool. " +
-          "See usage of " + AddCachePoolCommand.NAME + " for more details",
+          "See usage of " + AddCachePoolCommand.NAME + " for more details.",
           MAX_LINE_WIDTH) + "\n\n" +
           MAX_LINE_WIDTH) + "\n\n" +
           listing.toString();
           listing.toString();
     }
     }
@@ -651,9 +660,9 @@ public class CacheAdmin extends Configured implements Tool {
       String modeString = StringUtils.popOptionWithArgument("-mode", args);
       String modeString = StringUtils.popOptionWithArgument("-mode", args);
       Integer mode = (modeString == null) ?
       Integer mode = (modeString == null) ?
           null : Integer.parseInt(modeString, 8);
           null : Integer.parseInt(modeString, 8);
-      String weightString = StringUtils.popOptionWithArgument("-weight", args);
-      Integer weight = (weightString == null) ?
-          null : Integer.parseInt(weightString);
+      String limitString = StringUtils.popOptionWithArgument("-limit", args);
+      Long limit = (limitString == null) ?
+          null : Long.parseLong(limitString);
       String name = StringUtils.popFirstNonOption(args);
       String name = StringUtils.popFirstNonOption(args);
       if (name == null) {
       if (name == null) {
         System.err.println("You must specify a name when creating a " +
         System.err.println("You must specify a name when creating a " +
@@ -680,8 +689,8 @@ public class CacheAdmin extends Configured implements Tool {
         info.setMode(new FsPermission(mode.shortValue()));
         info.setMode(new FsPermission(mode.shortValue()));
         changed = true;
         changed = true;
       }
       }
-      if (weight != null) {
-        info.setWeight(weight);
+      if (limit != null) {
+        info.setLimit(limit);
         changed = true;
         changed = true;
       }
       }
       if (!changed) {
       if (!changed) {
@@ -709,8 +718,8 @@ public class CacheAdmin extends Configured implements Tool {
         System.out.print(prefix + "mode " + new FsPermission(mode.shortValue()));
         System.out.print(prefix + "mode " + new FsPermission(mode.shortValue()));
         prefix = " and ";
         prefix = " and ";
       }
       }
-      if (weight != null) {
-        System.out.print(prefix + "weight " + weight);
+      if (limit != null) {
+        System.out.print(prefix + "limit " + limit);
         prefix = " and ";
         prefix = " and ";
       }
       }
       System.out.print("\n");
       System.out.print("\n");
@@ -804,11 +813,12 @@ public class CacheAdmin extends Configured implements Tool {
           addField("OWNER", Justification.LEFT).
           addField("OWNER", Justification.LEFT).
           addField("GROUP", Justification.LEFT).
           addField("GROUP", Justification.LEFT).
           addField("MODE", Justification.LEFT).
           addField("MODE", Justification.LEFT).
-          addField("WEIGHT", Justification.RIGHT);
+          addField("LIMIT", Justification.RIGHT);
       if (printStats) {
       if (printStats) {
         builder.
         builder.
             addField("BYTES_NEEDED", Justification.RIGHT).
             addField("BYTES_NEEDED", Justification.RIGHT).
             addField("BYTES_CACHED", Justification.RIGHT).
             addField("BYTES_CACHED", Justification.RIGHT).
+            addField("BYTES_OVERLIMIT", Justification.RIGHT).
             addField("FILES_NEEDED", Justification.RIGHT).
             addField("FILES_NEEDED", Justification.RIGHT).
             addField("FILES_CACHED", Justification.RIGHT);
             addField("FILES_CACHED", Justification.RIGHT);
       }
       }
@@ -825,12 +835,19 @@ public class CacheAdmin extends Configured implements Tool {
             row.add(info.getOwnerName());
             row.add(info.getOwnerName());
             row.add(info.getGroupName());
             row.add(info.getGroupName());
             row.add(info.getMode() != null ? info.getMode().toString() : null);
             row.add(info.getMode() != null ? info.getMode().toString() : null);
-            row.add(
-                info.getWeight() != null ? info.getWeight().toString() : null);
+            Long limit = info.getLimit();
+            String limitString;
+            if (limit != null && limit.equals(CachePool.DEFAULT_LIMIT)) {
+              limitString = "unlimited";
+            } else {
+              limitString = "" + limit;
+            }
+            row.add(limitString);
             if (printStats) {
             if (printStats) {
               CachePoolStats stats = entry.getStats();
               CachePoolStats stats = entry.getStats();
               row.add(Long.toString(stats.getBytesNeeded()));
               row.add(Long.toString(stats.getBytesNeeded()));
               row.add(Long.toString(stats.getBytesCached()));
               row.add(Long.toString(stats.getBytesCached()));
+              row.add(Long.toString(stats.getBytesOverlimit()));
               row.add(Long.toString(stats.getFilesNeeded()));
               row.add(Long.toString(stats.getFilesNeeded()));
               row.add(Long.toString(stats.getFilesCached()));
               row.add(Long.toString(stats.getFilesCached()));
             }
             }

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

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

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

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

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

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

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

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

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

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

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

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

+ 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
  * This test simulates a variety of situations when blocks are being
  * intentionally corrupted, unexpectedly modified, and so on before a block
  * 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 class TestBlockReport {
   public static final Log LOG = LogFactory.getLog(TestBlockReport.class);
   public static final Log LOG = LogFactory.getLog(TestBlockReport.class);
@@ -157,6 +165,113 @@ public class TestBlockReport {
     return reports;
     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
    * Test write a file, verifies and closes it. Then the length of the blocks
    * are messed up and BlockReport is forced.
    * are messed up and BlockReport is forced.
@@ -164,8 +279,7 @@ public class TestBlockReport {
    *
    *
    * @throws java.io.IOException on an error
    * @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();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
 
 
@@ -198,7 +312,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
 
 
     List<LocatedBlock> blocksAfterReport =
     List<LocatedBlock> blocksAfterReport =
       DFSTestUtil.getAllBlocks(fs.open(filePath));
       DFSTestUtil.getAllBlocks(fs.open(filePath));
@@ -224,8 +338,7 @@ public class TestBlockReport {
    *
    *
    * @throws IOException in case of errors
    * @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();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     LOG.info("Running test " + METHOD_NAME);
     LOG.info("Running test " + METHOD_NAME);
 
 
@@ -280,7 +393,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn0.getDNRegistrationForBP(poolId);
     DatanodeRegistration dnR = dn0.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn0, poolId, false, false);
     StorageBlockReport[] reports = getBlockReports(dn0, poolId, false, false);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
 
 
     BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
     BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
         .getBlockManager());
         .getBlockManager());
@@ -301,8 +414,7 @@ public class TestBlockReport {
    *
    *
    * @throws IOException in case of an error
    * @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();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     ArrayList<Block> blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath);
     ArrayList<Block> blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath);
@@ -312,11 +424,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
     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();
     printStats();
 
 
     assertThat("Wrong number of corrupt blocks",
     assertThat("Wrong number of corrupt blocks",
@@ -333,8 +441,7 @@ public class TestBlockReport {
    *
    *
    * @throws IOException in case of an error
    * @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();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     DFSTestUtil.createFile(fs, filePath,
     DFSTestUtil.createFile(fs, filePath,
@@ -352,11 +459,7 @@ public class TestBlockReport {
 
 
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
     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();
     printStats();
 
 
     assertThat("Wrong number of corrupt blocks",
     assertThat("Wrong number of corrupt blocks",
@@ -373,8 +476,7 @@ public class TestBlockReport {
    *
    *
    * @throws IOException in case of an error
    * @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();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     final int DN_N1 = DN_N0 + 1;
     final int DN_N1 = DN_N0 + 1;
@@ -387,7 +489,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
     printStats();
     printStats();
     assertEquals("Wrong number of PendingReplication Blocks",
     assertEquals("Wrong number of PendingReplication Blocks",
       0, cluster.getNamesystem().getUnderReplicatedBlocks());
       0, cluster.getNamesystem().getUnderReplicatedBlocks());
@@ -406,8 +508,7 @@ public class TestBlockReport {
    *
    *
    * @throws IOException in case of an error
    * @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();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     final int DN_N1 = DN_N0 + 1;
     final int DN_N1 = DN_N0 + 1;
@@ -421,7 +522,7 @@ public class TestBlockReport {
     String poolId = cluster.getNamesystem().getBlockPoolId();
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
     StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
     printStats();
     printStats();
 
 
     assertThat("Wrong number of corrupt blocks",
     assertThat("Wrong number of corrupt blocks",
@@ -432,7 +533,7 @@ public class TestBlockReport {
                cluster.getNamesystem().getPendingReplicationBlocks(), is(0L));
                cluster.getNamesystem().getPendingReplicationBlocks(), is(0L));
 
 
     reports = getBlockReports(dn, poolId, true, true);
     reports = getBlockReports(dn, poolId, true, true);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    sendBlockReports(dnR, poolId, reports, splitBlockReports);
     printStats();
     printStats();
 
 
     assertThat("Wrong number of corrupt blocks",
     assertThat("Wrong number of corrupt blocks",
@@ -458,8 +559,7 @@ public class TestBlockReport {
    *
    *
    * @throws IOException in case of an error
    * @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();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     final int DN_N1 = DN_N0 + 1;
     final int DN_N1 = DN_N0 + 1;
@@ -483,8 +583,8 @@ public class TestBlockReport {
       DataNode dn = cluster.getDataNodes().get(DN_N1);
       DataNode dn = cluster.getDataNodes().get(DN_N1);
       String poolId = cluster.getNamesystem().getBlockPoolId();
       String poolId = cluster.getNamesystem().getBlockPoolId();
       DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
       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();
       printStats();
       assertEquals("Wrong number of PendingReplication blocks",
       assertEquals("Wrong number of PendingReplication blocks",
         blocks.size(), cluster.getNamesystem().getPendingReplicationBlocks());
         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
   // Similar to BlockReport_08 but corrupts GS and len of the TEMPORARY's
   // replica block. Expect the same behaviour: NN should simply ignore this
   // replica block. Expect the same behaviour: NN should simply ignore this
   // block
   // block
-  @Test
-  public void blockReport_09() throws IOException {
+  private void blockReport_09(boolean splitBlockReports) throws IOException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     final int DN_N1 = DN_N0 + 1;
     final int DN_N1 = DN_N0 + 1;
@@ -526,8 +625,8 @@ public class TestBlockReport {
       DataNode dn = cluster.getDataNodes().get(DN_N1);
       DataNode dn = cluster.getDataNodes().get(DN_N1);
       String poolId = cluster.getNamesystem().getBlockPoolId();
       String poolId = cluster.getNamesystem().getBlockPoolId();
       DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
       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();
       printStats();
       assertEquals("Wrong number of PendingReplication blocks",
       assertEquals("Wrong number of PendingReplication blocks",
         2, cluster.getNamesystem().getPendingReplicationBlocks());
         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")
         .setOwnerName("carlton")
         .setGroupName("party")
         .setGroupName("party")
         .setMode(new FsPermission((short)0700))
         .setMode(new FsPermission((short)0700))
-        .setWeight(1989));
+        .setLimit(1989l));
     // OP_ADD_PATH_BASED_CACHE_DIRECTIVE 33
     // OP_ADD_PATH_BASED_CACHE_DIRECTIVE 33
     long id = dfs.addCacheDirective(
     long id = dfs.addCacheDirective(
         new CacheDirectiveInfo.Builder().
         new CacheDirectiveInfo.Builder().

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

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

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

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

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

+ 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.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 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.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.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.log4j.Level;
 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.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
@@ -780,7 +781,7 @@ public class TestINodeFile {
       }
       }
       System.out.println("Adding component " + DFSUtil.bytes2String(component));
       System.out.println("Adding component " + DFSUtil.bytes2String(component));
       dir = new INodeDirectory(++id, component, permstatus, 0);
       dir = new INodeDirectory(++id, component, permstatus, 0);
-      prev.addChild(dir, false, null, null);
+      prev.addChild(dir, false, null);
       prev = dir;
       prev = dir;
     }
     }
     return dir; // Last Inode in the chain
     return dir; // Last Inode in the chain
@@ -921,6 +922,7 @@ public class TestINodeFile {
   public void testDotdotInodePath() throws Exception {
   public void testDotdotInodePath() throws Exception {
     final Configuration conf = new Configuration();
     final Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
+    DFSClient client = null;
     try {
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       cluster.waitActive();
       cluster.waitActive();
@@ -933,7 +935,7 @@ public class TestINodeFile {
       long parentId = fsdir.getINode("/").getId();
       long parentId = fsdir.getINode("/").getId();
       String testPath = "/.reserved/.inodes/" + dirId + "/..";
       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);
       HdfsFileStatus status = client.getFileInfo(testPath);
       assertTrue(parentId == status.getFileId());
       assertTrue(parentId == status.getFileId());
       
       
@@ -943,6 +945,7 @@ public class TestINodeFile {
       assertTrue(parentId == status.getFileId());
       assertTrue(parentId == status.getFileId());
       
       
     } finally {
     } finally {
+      IOUtils.cleanup(LOG, client);
       if (cluster != null) {
       if (cluster != null) {
         cluster.shutdown();
         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.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -206,8 +205,7 @@ public class TestSnapshotPathINodes {
     // Check the INode for file1 (snapshot file)
     // Check the INode for file1 (snapshot file)
     INode snapshotFileNode = inodes[inodes.length - 1]; 
     INode snapshotFileNode = inodes[inodes.length - 1]; 
     assertINodeFile(snapshotFileNode, file1);
     assertINodeFile(snapshotFileNode, file1);
-    assertTrue(snapshotFileNode.getParent() instanceof 
-        INodeDirectoryWithSnapshot);
+    assertTrue(snapshotFileNode.getParent().isWithSnapshot());
     
     
     // Call getExistingPathINodes and request only one INode.
     // Call getExistingPathINodes and request only one INode.
     nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 1, false);
     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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
@@ -760,7 +761,7 @@ public class TestRetryCacheWithHA {
 
 
     @Override
     @Override
     void invoke() throws Exception {
     void invoke() throws Exception {
-      result = client.addCacheDirective(directive);
+      result = client.addCacheDirective(directive, EnumSet.of(CacheFlag.FORCE));
     }
     }
 
 
     @Override
     @Override
@@ -802,7 +803,7 @@ public class TestRetryCacheWithHA {
     @Override
     @Override
     void prepare() throws Exception {
     void prepare() throws Exception {
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
-      id = client.addCacheDirective(directive);
+      id = client.addCacheDirective(directive, EnumSet.of(CacheFlag.FORCE));
     }
     }
 
 
     @Override
     @Override
@@ -811,7 +812,7 @@ public class TestRetryCacheWithHA {
           new CacheDirectiveInfo.Builder().
           new CacheDirectiveInfo.Builder().
               setId(id).
               setId(id).
               setReplication(newReplication).
               setReplication(newReplication).
-              build());
+              build(), EnumSet.of(CacheFlag.FORCE));
     }
     }
 
 
     @Override
     @Override
@@ -858,7 +859,7 @@ public class TestRetryCacheWithHA {
     @Override
     @Override
     void prepare() throws Exception {
     void prepare() throws Exception {
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
       dfs.addCachePool(new CachePoolInfo(directive.getPool()));
-      id = dfs.addCacheDirective(directive);
+      id = dfs.addCacheDirective(directive, EnumSet.of(CacheFlag.FORCE));
     }
     }
 
 
     @Override
     @Override
@@ -936,19 +937,19 @@ public class TestRetryCacheWithHA {
 
 
     @Override
     @Override
     void prepare() throws Exception {
     void prepare() throws Exception {
-      client.addCachePool(new CachePoolInfo(pool).setWeight(10));
+      client.addCachePool(new CachePoolInfo(pool).setLimit(10l));
     }
     }
 
 
     @Override
     @Override
     void invoke() throws Exception {
     void invoke() throws Exception {
-      client.modifyCachePool(new CachePoolInfo(pool).setWeight(99));
+      client.modifyCachePool(new CachePoolInfo(pool).setLimit(99l));
     }
     }
 
 
     @Override
     @Override
     boolean checkNamenodeBeforeReturn() throws Exception {
     boolean checkNamenodeBeforeReturn() throws Exception {
       for (int i = 0; i < CHECKTIMES; i++) {
       for (int i = 0; i < CHECKTIMES; i++) {
         RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
         RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
-        if (iter.hasNext() && iter.next().getInfo().getWeight() == 99) {
+        if (iter.hasNext() && (long)iter.next().getInfo().getLimit() == 99) {
           return true;
           return true;
         }
         }
         Thread.sleep(1000);
         Thread.sleep(1000);
@@ -1216,7 +1217,7 @@ public class TestRetryCacheWithHA {
       CacheDirectiveInfo directiveInfo =
       CacheDirectiveInfo directiveInfo =
         new CacheDirectiveInfo.Builder().setPool(poolName).setPath(path).build();
         new CacheDirectiveInfo.Builder().setPool(poolName).setPath(path).build();
       dfs.addCachePool(new CachePoolInfo(poolName));
       dfs.addCachePool(new CachePoolInfo(poolName));
-      dfs.addCacheDirective(directiveInfo);
+      dfs.addCacheDirective(directiveInfo, EnumSet.of(CacheFlag.FORCE));
       poolNames.add(poolName);
       poolNames.add(poolName);
     }
     }
     listCacheDirectives(poolNames, 0);
     listCacheDirectives(poolNames, 0);

+ 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.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 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.apache.log4j.Level;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 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();
     FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
     INode subNode = fsdir.getINode(sub.toString());
     INode subNode = fsdir.getINode(sub.toString());
-    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
+    assertTrue(subNode.asDirectory().isWithSnapshot());
     
     
     hdfs.allowSnapshot(sub);
     hdfs.allowSnapshot(sub);
     subNode = fsdir.getINode(sub.toString());
     subNode = fsdir.getINode(sub.toString());
@@ -366,6 +366,6 @@ public class TestNestedSnapshots {
     
     
     hdfs.disallowSnapshot(sub);
     hdfs.disallowSnapshot(sub);
     subNode = fsdir.getINode(sub.toString());
     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.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 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;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 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.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.test.GenericTestUtils;
 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
     // only 2 references: one in deleted list of sdir1, one in created list of
     // sdir1
     // sdir1
     assertEquals(2, fooWithCount.getReferenceCount());
     assertEquals(2, fooWithCount.getReferenceCount());
-    INodeDirectoryWithSnapshot foo = (INodeDirectoryWithSnapshot) fooWithCount
-        .asDirectory();
+    INodeDirectory foo = fooWithCount.asDirectory();
     assertEquals(1, foo.getDiffs().asList().size());
     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();
     INodeFile bar1 = fsdir.getINode4Write(bar1_dir1.toString()).asFile();
     assertEquals(1, bar1.getDiffs().asList().size());
     assertEquals(1, bar1.getDiffs().asList().size());
     assertEquals("s1", bar1.getDiffs().getLastSnapshot().getRoot()
     assertEquals("s1", bar1.getDiffs().getLastSnapshot().getRoot()
@@ -973,8 +972,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount fooWithCount = (WithCount) fooRef.getReferredINode();
     INodeReference.WithCount fooWithCount = (WithCount) fooRef.getReferredINode();
     // 5 references: s1, s22, s333, s2222, current tree of sdir1
     // 5 references: s1, s22, s333, s2222, current tree of sdir1
     assertEquals(5, fooWithCount.getReferenceCount());
     assertEquals(5, fooWithCount.getReferenceCount());
-    INodeDirectoryWithSnapshot foo = (INodeDirectoryWithSnapshot) fooWithCount
-        .asDirectory();
+    INodeDirectory foo = fooWithCount.asDirectory();
     List<DirectoryDiff> fooDiffs = foo.getDiffs().asList();
     List<DirectoryDiff> fooDiffs = foo.getDiffs().asList();
     assertEquals(4, fooDiffs.size());
     assertEquals(4, fooDiffs.size());
     assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
     assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1032,7 +1030,7 @@ public class TestRenameWithSnapshots {
     fooRef = fsdir.getINode(foo_s2222.toString()).asReference();
     fooRef = fsdir.getINode(foo_s2222.toString()).asReference();
     fooWithCount = (WithCount) fooRef.getReferredINode();
     fooWithCount = (WithCount) fooRef.getReferredINode();
     assertEquals(4, fooWithCount.getReferenceCount());
     assertEquals(4, fooWithCount.getReferenceCount());
-    foo = (INodeDirectoryWithSnapshot) fooWithCount.asDirectory();
+    foo = fooWithCount.asDirectory();
     fooDiffs = foo.getDiffs().asList();
     fooDiffs = foo.getDiffs().asList();
     assertEquals(4, fooDiffs.size());
     assertEquals(4, fooDiffs.size());
     assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
     assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1171,8 +1169,7 @@ public class TestRenameWithSnapshots {
     assertTrue(fooRef instanceof INodeReference.WithName);
     assertTrue(fooRef instanceof INodeReference.WithName);
     INodeReference.WithCount fooWC = (WithCount) fooRef.getReferredINode();
     INodeReference.WithCount fooWC = (WithCount) fooRef.getReferredINode();
     assertEquals(1, fooWC.getReferenceCount());
     assertEquals(1, fooWC.getReferenceCount());
-    INodeDirectoryWithSnapshot fooDir = (INodeDirectoryWithSnapshot) fooWC
-        .getReferredINode().asDirectory();
+    INodeDirectory fooDir = fooWC.getReferredINode().asDirectory();
     List<DirectoryDiff> diffs = fooDir.getDiffs().asList();
     List<DirectoryDiff> diffs = fooDir.getDiffs().asList();
     assertEquals(1, diffs.size());
     assertEquals(1, diffs.size());
     assertEquals("s2", diffs.get(0).snapshot.getRoot().getLocalName());
     assertEquals("s2", diffs.get(0).snapshot.getRoot().getLocalName());
@@ -1263,7 +1260,7 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory mockDir2 = spy(dir2);
     INodeDirectory mockDir2 = spy(dir2);
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject(), (INodeMap) anyObject());
+            (Snapshot) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     
     
@@ -1288,9 +1285,8 @@ public class TestRenameWithSnapshots {
     assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
     assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
     
     
     INode fooNode = fsdir.getINode4Write(foo.toString());
     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(1, fooDiffs.size());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     
     
@@ -1302,7 +1298,7 @@ public class TestRenameWithSnapshots {
     assertFalse(hdfs.exists(newfoo));
     assertFalse(hdfs.exists(newfoo));
     INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
     INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
         .asDirectory();
-    assertFalse(dir2Node instanceof INodeDirectoryWithSnapshot);
+    assertFalse(dir2Node.isWithSnapshot());
     ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
     ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
     assertEquals(1, dir2Children.size());
     assertEquals(1, dir2Children.size());
     assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
     assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
@@ -1331,7 +1327,7 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory mockDir2 = spy(dir2);
     INodeDirectory mockDir2 = spy(dir2);
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject(), (INodeMap) anyObject());
+            (Snapshot) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     
     
@@ -1366,7 +1362,7 @@ public class TestRenameWithSnapshots {
     assertFalse(hdfs.exists(newfoo));
     assertFalse(hdfs.exists(newfoo));
     INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
     INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
         .asDirectory();
-    assertFalse(dir2Node instanceof INodeDirectoryWithSnapshot);
+    assertFalse(dir2Node.isWithSnapshot());
     ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
     ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
     assertEquals(1, dir2Children.size());
     assertEquals(1, dir2Children.size());
     assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
     assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
@@ -1393,7 +1389,7 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
     INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
     INodeDirectory mockDir3 = spy(dir3);
     INodeDirectory mockDir3 = spy(dir3);
     doReturn(false).when(mockDir3).addChild((INode) anyObject(), anyBoolean(),
     doReturn(false).when(mockDir3).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject(), (INodeMap) anyObject());
+            (Snapshot) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     
     
@@ -1420,8 +1416,7 @@ public class TestRenameWithSnapshots {
     INode fooNode = fsdir.getINode4Write(foo_dir2.toString());
     INode fooNode = fsdir.getINode4Write(foo_dir2.toString());
     assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
     assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
     assertTrue(fooNode instanceof INodeReference.DstReference);
     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(1, fooDiffs.size());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     
     
@@ -1455,8 +1450,7 @@ public class TestRenameWithSnapshots {
     assertTrue(hdfs.exists(foo_s3));
     assertTrue(hdfs.exists(foo_s3));
     
     
     assertTrue(fooNode instanceof INodeReference.DstReference);
     assertTrue(fooNode instanceof INodeReference.DstReference);
-    fooDiffs = ((INodeDirectoryWithSnapshot) fooNode.asDirectory()).getDiffs()
-        .asList();
+    fooDiffs = fooNode.asDirectory().getDiffs().asList();
     assertEquals(2, fooDiffs.size());
     assertEquals(2, fooDiffs.size());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     assertEquals("s3", fooDiffs.get(1).snapshot.getRoot().getLocalName());
     assertEquals("s3", fooDiffs.get(1).snapshot.getRoot().getLocalName());
@@ -1495,10 +1489,9 @@ public class TestRenameWithSnapshots {
     INodeDirectory mockDir3 = spy(dir3);
     INodeDirectory mockDir3 = spy(dir3);
     // fail the rename but succeed in undo
     // fail the rename but succeed in undo
     doReturn(false).when(mockDir3).addChild((INode) Mockito.isNull(),
     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();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     foo3Node.setParent(mockDir3);
     foo3Node.setParent(mockDir3);
@@ -1561,7 +1554,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(null));
         .getChildrenList(null));
     assertEquals(1, childrenList.size());
     assertEquals(1, childrenList.size());
     INode fooNode = childrenList.get(0);
     INode fooNode = childrenList.get(0);
-    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    assertTrue(fooNode.asDirectory().isWithSnapshot());
     INode barNode = fsdir.getINode4Write(bar.toString());
     INode barNode = fsdir.getINode4Write(bar.toString());
     assertTrue(barNode.getClass() == INodeFile.class);
     assertTrue(barNode.getClass() == INodeFile.class);
     assertSame(fooNode, barNode.getParent());
     assertSame(fooNode, barNode.getParent());
@@ -1637,7 +1630,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(null));
         .getChildrenList(null));
     assertEquals(1, childrenList.size());
     assertEquals(1, childrenList.size());
     INode fooNode = childrenList.get(0);
     INode fooNode = childrenList.get(0);
-    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    assertTrue(fooNode.asDirectory().isWithSnapshot());
     assertSame(dir1Node, fooNode.getParent());
     assertSame(dir1Node, fooNode.getParent());
     List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
     List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
         .getDiffs().asList();
         .getDiffs().asList();
@@ -1656,7 +1649,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(null));
         .getChildrenList(null));
     assertEquals(1, childrenList.size());
     assertEquals(1, childrenList.size());
     INode subdir2Node = childrenList.get(0);
     INode subdir2Node = childrenList.get(0);
-    assertTrue(subdir2Node.getClass() == INodeDirectoryWithSnapshot.class);
+    assertTrue(subdir2Node.asDirectory().isWithSnapshot());
     assertSame(dir2Node, subdir2Node.getParent());
     assertSame(dir2Node, subdir2Node.getParent());
     assertSame(subdir2Node, fsdir.getINode4Write(sub_dir2.toString()));
     assertSame(subdir2Node, fsdir.getINode4Write(sub_dir2.toString()));
     INode subsubdir2Node = fsdir.getINode4Write(subsub_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.CREATED).isEmpty());
     assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
     assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
     
     
-    diffList = ((INodeDirectoryWithSnapshot) subdir2Node).getDiffs().asList();
+    diffList = subdir2Node.asDirectory().getDiffs().asList();
     assertEquals(0, diffList.size());
     assertEquals(0, diffList.size());
   }
   }
   
   
@@ -1697,8 +1690,7 @@ public class TestRenameWithSnapshots {
     }
     }
     
     
     // check
     // check
-    INodeDirectoryWithSnapshot fooNode = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(foo.toString());
+    INodeDirectory fooNode = fsdir.getINode4Write(foo.toString()).asDirectory();
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     assertEquals(1, children.size());
     assertEquals(1, children.size());
     List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
     List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
@@ -1948,8 +1940,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount wc = 
     INodeReference.WithCount wc = 
         (WithCount) fooRef.asReference().getReferredINode();
         (WithCount) fooRef.asReference().getReferredINode();
     assertEquals(1, wc.getReferenceCount());
     assertEquals(1, wc.getReferenceCount());
-    INodeDirectoryWithSnapshot fooNode = 
-        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    INodeDirectory fooNode = wc.getReferredINode().asDirectory();
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     assertEquals(1, children.size());
     assertEquals(1, children.size());
     assertEquals(bar.getName(), children.get(0).getLocalName());
     assertEquals(bar.getName(), children.get(0).getLocalName());
@@ -2017,8 +2008,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount wc = 
     INodeReference.WithCount wc = 
         (WithCount) fooRef.asReference().getReferredINode();
         (WithCount) fooRef.asReference().getReferredINode();
     assertEquals(2, wc.getReferenceCount());
     assertEquals(2, wc.getReferenceCount());
-    INodeDirectoryWithSnapshot fooNode = 
-        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    INodeDirectory fooNode = wc.getReferredINode().asDirectory();
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     assertEquals(3, children.size());
     assertEquals(3, children.size());
     assertEquals(bar.getName(), children.get(0).getLocalName());
     assertEquals(bar.getName(), children.get(0).getLocalName());
@@ -2044,9 +2034,9 @@ public class TestRenameWithSnapshots {
   
   
   /**
   /**
    * This test demonstrates that 
    * This test demonstrates that 
-   * {@link INodeDirectoryWithSnapshot#removeChild(INode, Snapshot, INodeMap)}
+   * {@link INodeDirectory#removeChild(INode, Snapshot)}
    * and 
    * and 
-   * {@link INodeDirectoryWithSnapshot#addChild(INode, boolean, Snapshot, INodeMap)}
+   * {@link INodeDirectory#addChild(INode, boolean, Snapshot)}
    * should use {@link INode#isInLatestSnapshot(Snapshot)} to check if the 
    * should use {@link INode#isInLatestSnapshot(Snapshot)} to check if the 
    * added/removed child should be recorded in snapshots.
    * added/removed child should be recorded in snapshots.
    */
    */
@@ -2063,7 +2053,7 @@ public class TestRenameWithSnapshots {
     hdfs.mkdirs(foo);
     hdfs.mkdirs(foo);
     SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
     SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
     final Path bar = new Path(foo, "bar");
     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);
     DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
     // delete snapshot s1. now foo is not in any snapshot
     // delete snapshot s1. now foo is not in any snapshot
     hdfs.deleteSnapshot(dir1, "s1");
     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 
     // 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
     // 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.
     // stored in the deleted list of foo, and will be destroyed.
     hdfs.delete(foo2, true);
     hdfs.delete(foo2, true);
     
     
@@ -2130,8 +2120,8 @@ public class TestRenameWithSnapshots {
     // check the internal
     // check the internal
     assertFalse("after deleting s0, " + foo_s0 + " should not exist",
     assertFalse("after deleting s0, " + foo_s0 + " should not exist",
         hdfs.exists(foo_s0));
         hdfs.exists(foo_s0));
-    INodeDirectoryWithSnapshot dir2Node = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(dir2.toString());
+    INodeDirectory dir2Node = fsdir.getINode4Write(dir2.toString())
+        .asDirectory();
     assertTrue("the diff list of " + dir2
     assertTrue("the diff list of " + dir2
         + " should be empty after deleting s0", dir2Node.getDiffs().asList()
         + " should be empty after deleting s0", dir2Node.getDiffs().asList()
         .isEmpty());
         .isEmpty());
@@ -2140,16 +2130,14 @@ public class TestRenameWithSnapshots {
     INode fooRefNode = fsdir.getINode4Write(newfoo.toString());
     INode fooRefNode = fsdir.getINode4Write(newfoo.toString());
     assertTrue(fooRefNode instanceof INodeReference.DstReference);
     assertTrue(fooRefNode instanceof INodeReference.DstReference);
     INodeDirectory fooNode = fooRefNode.asDirectory();
     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
     // 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();
     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 
     // list and children list are empty 
-    assertTrue(((INodeDirectoryWithSnapshot) barNode).getDiffs().asList()
-        .isEmpty());
+    assertTrue(barNode.getDiffs().asList().isEmpty());
     assertTrue(barNode.getChildrenList(null).isEmpty());
     assertTrue(barNode.getChildrenList(null).isEmpty());
     
     
     restartClusterAndCheckImage(true);
     restartClusterAndCheckImage(true);
@@ -2199,8 +2187,8 @@ public class TestRenameWithSnapshots {
     assertTrue(hdfs.exists(file_s0));
     assertTrue(hdfs.exists(file_s0));
     
     
     // check dir1: foo should be in the created list of 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();
     List<DirectoryDiff> dir1DiffList = dir1Node.getDiffs().asList();
     assertEquals(1, dir1DiffList.size());
     assertEquals(1, dir1DiffList.size());
     List<INode> dList = dir1DiffList.get(0).getChildrenDiff()
     List<INode> dList = dir1DiffList.get(0).getChildrenDiff()
@@ -2215,8 +2203,8 @@ public class TestRenameWithSnapshots {
     
     
     // check foo and its subtree
     // check foo and its subtree
     final Path newbar = new Path(newfoo, bar.getName());
     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());
     assertSame(fooNode.asDirectory(), barNode.getParent());
     // bar should only have a snapshot diff for s0
     // bar should only have a snapshot diff for s0
     List<DirectoryDiff> barDiffList = barNode.getDiffs().asList();
     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
     // check dir2: a WithName instance for foo should be in the deleted list
     // of the snapshot diff for s2
     // 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();
     List<DirectoryDiff> dir2DiffList = dir2Node.getDiffs().asList();
     // dir2Node should contain 2 snapshot diffs, one for s2, and the other was
     // dir2Node should contain 2 snapshot diffs, one for s2, and the other was
     // originally s1 (created when dir2 was transformed to a snapshottable dir),
     // 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 
     // make sure the file under bar is deleted 
     final Path barInS0 = SnapshotTestHelper.getSnapshotPath(test, "s0",
     final Path barInS0 = SnapshotTestHelper.getSnapshotPath(test, "s0",
         "foo/bar");
         "foo/bar");
-    INodeDirectoryWithSnapshot barNode = (INodeDirectoryWithSnapshot) fsdir
-        .getINode(barInS0.toString());
+    INodeDirectory barNode = fsdir.getINode(barInS0.toString()).asDirectory();
     assertEquals(0, barNode.getChildrenList(null).size());
     assertEquals(0, barNode.getChildrenList(null).size());
     List<DirectoryDiff> diffList = barNode.getDiffs().asList();
     List<DirectoryDiff> diffList = barNode.getDiffs().asList();
     assertEquals(1, diffList.size());
     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.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 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.apache.hadoop.hdfs.util.Diff.ListType;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
@@ -92,12 +92,12 @@ public class TestSetQuotaWithSnapshot {
     INodeDirectory subNode = INodeDirectory.valueOf(
     INodeDirectory subNode = INodeDirectory.valueOf(
         fsdir.getINode(sub.toString()), sub);
         fsdir.getINode(sub.toString()), sub);
     // subNode should be a INodeDirectory, but not an INodeDirectoryWithSnapshot
     // 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);
     hdfs.setQuota(sub, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
     subNode = INodeDirectory.valueOf(fsdir.getINode(sub.toString()), sub);
     subNode = INodeDirectory.valueOf(fsdir.getINode(sub.toString()), sub);
     assertTrue(subNode.isQuotaSet());
     assertTrue(subNode.isQuotaSet());
-    assertFalse(subNode instanceof INodeDirectoryWithSnapshot);
+    assertFalse(subNode.isWithSnapshot());
   }
   }
   
   
   /**
   /**
@@ -150,8 +150,8 @@ public class TestSetQuotaWithSnapshot {
     DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
     DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
     hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
     hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
     INode subNode = fsdir.getINode4Write(subDir.toString());
     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(1, diffList.size());
     assertEquals("s2", Snapshot.getSnapshotName(diffList.get(0).snapshot));
     assertEquals("s2", Snapshot.getSnapshotName(diffList.get(0).snapshot));
     List<INode> createdList = diffList.get(0).getChildrenDiff().getList(ListType.CREATED);
     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.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 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.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 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
     // make sure the whole subtree of sub is stored correctly in snapshot
     Path snapshotSub = SnapshotTestHelper.getSnapshotPath(dir, "s1",
     Path snapshotSub = SnapshotTestHelper.getSnapshotPath(dir, "s1",
         sub.getName());
         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.
     // the snapshot copy of sub has only one child subsub.
     // newFile should have been destroyed
     // newFile should have been destroyed
     assertEquals(1, snapshotNode4Sub.getChildrenList(null).size());
     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
     // check the snapshot copy of subsub, which is contained in the subtree of
     // sub's snapshot copy
     // sub's snapshot copy
     INode snapshotNode4Subsub = snapshotNode4Sub.getChildrenList(null).get(0);
     INode snapshotNode4Subsub = snapshotNode4Sub.getChildrenList(null).get(0);
-    assertEquals(INodeDirectoryWithSnapshot.class,
-        snapshotNode4Subsub.getClass());
+    assertTrue(snapshotNode4Subsub.asDirectory().isWithSnapshot());
     assertTrue(snapshotNode4Sub == snapshotNode4Subsub.getParent());
     assertTrue(snapshotNode4Sub == snapshotNode4Subsub.getParent());
     // check the children of subsub
     // check the children of subsub
     INodeDirectory snapshotSubsubDir = (INodeDirectory) snapshotNode4Subsub;
     INodeDirectory snapshotSubsubDir = (INodeDirectory) snapshotNode4Subsub;
@@ -478,8 +477,8 @@ public class TestSnapshotDeletion {
     DirectoryDiffList diffList = dirNode.getDiffs();
     DirectoryDiffList diffList = dirNode.getDiffs();
     assertEquals(1, diffList.asList().size());
     assertEquals(1, diffList.asList().size());
     assertEquals("s1", diffList.getLast().snapshot.getRoot().getLocalName());
     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());
     assertEquals(0, diffList.asList().size());
     
     
     // check 2. noChangeDir and noChangeFile are still there
     // 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.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 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.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
 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>
       <TXID>2</TXID>
       <DELEGATION_KEY>
       <DELEGATION_KEY>
         <KEY_ID>1</KEY_ID>
         <KEY_ID>1</KEY_ID>
-        <EXPIRY_DATE>1386695013416</EXPIRY_DATE>
-        <KEY>360a10c6ecac725e</KEY>
+        <EXPIRY_DATE>1387701670577</EXPIRY_DATE>
+        <KEY>7bb5467995769b59</KEY>
       </DELEGATION_KEY>
       </DELEGATION_KEY>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -24,8 +24,8 @@
       <TXID>3</TXID>
       <TXID>3</TXID>
       <DELEGATION_KEY>
       <DELEGATION_KEY>
         <KEY_ID>2</KEY_ID>
         <KEY_ID>2</KEY_ID>
-        <EXPIRY_DATE>1386695013425</EXPIRY_DATE>
-        <KEY>9b110c0b83225f7d</KEY>
+        <EXPIRY_DATE>1387701670580</EXPIRY_DATE>
+        <KEY>a5a3a2755e36827b</KEY>
       </DELEGATION_KEY>
       </DELEGATION_KEY>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -37,17 +37,17 @@
       <INODEID>16386</INODEID>
       <INODEID>16386</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814612</MTIME>
-      <ATIME>1386003814612</ATIME>
+      <MTIME>1387010471220</MTIME>
+      <ATIME>1387010471220</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>7</RPC_CALLID>
       <RPC_CALLID>7</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -59,13 +59,13 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814665</MTIME>
-      <ATIME>1386003814612</ATIME>
+      <MTIME>1387010471276</MTIME>
+      <ATIME>1387010471220</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -78,8 +78,8 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <SRC>/file_create_u\0001;F431</SRC>
       <SRC>/file_create_u\0001;F431</SRC>
       <DST>/file_moved</DST>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1386003814671</TIMESTAMP>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <TIMESTAMP>1387010471286</TIMESTAMP>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>9</RPC_CALLID>
       <RPC_CALLID>9</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -89,8 +89,8 @@
       <TXID>7</TXID>
       <TXID>7</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
       <PATH>/file_moved</PATH>
-      <TIMESTAMP>1386003814678</TIMESTAMP>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <TIMESTAMP>1387010471299</TIMESTAMP>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>10</RPC_CALLID>
       <RPC_CALLID>10</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -101,9 +101,9 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>16387</INODEID>
       <INODEID>16387</INODEID>
       <PATH>/directory_mkdir</PATH>
       <PATH>/directory_mkdir</PATH>
-      <TIMESTAMP>1386003814686</TIMESTAMP>
+      <TIMESTAMP>1387010471312</TIMESTAMP>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>493</MODE>
         <MODE>493</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -136,7 +136,7 @@
       <TXID>12</TXID>
       <TXID>12</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>15</RPC_CALLID>
       <RPC_CALLID>15</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -147,7 +147,7 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>16</RPC_CALLID>
       <RPC_CALLID>16</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -157,7 +157,7 @@
       <TXID>14</TXID>
       <TXID>14</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>17</RPC_CALLID>
       <RPC_CALLID>17</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -169,17 +169,17 @@
       <INODEID>16388</INODEID>
       <INODEID>16388</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814712</MTIME>
-      <ATIME>1386003814712</ATIME>
+      <MTIME>1387010471373</MTIME>
+      <ATIME>1387010471373</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>18</RPC_CALLID>
       <RPC_CALLID>18</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -191,13 +191,13 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814714</MTIME>
-      <ATIME>1386003814712</ATIME>
+      <MTIME>1387010471380</MTIME>
+      <ATIME>1387010471373</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -253,9 +253,9 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <SRC>/file_create_u\0001;F431</SRC>
       <SRC>/file_create_u\0001;F431</SRC>
       <DST>/file_moved</DST>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1386003814732</TIMESTAMP>
+      <TIMESTAMP>1387010471428</TIMESTAMP>
       <OPTIONS>NONE</OPTIONS>
       <OPTIONS>NONE</OPTIONS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>25</RPC_CALLID>
       <RPC_CALLID>25</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -267,17 +267,17 @@
       <INODEID>16389</INODEID>
       <INODEID>16389</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814737</MTIME>
-      <ATIME>1386003814737</ATIME>
+      <MTIME>1387010471438</MTIME>
+      <ATIME>1387010471438</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>27</RPC_CALLID>
       <RPC_CALLID>27</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -388,8 +388,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814889</MTIME>
-      <ATIME>1386003814737</ATIME>
+      <MTIME>1387010471540</MTIME>
+      <ATIME>1387010471438</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -409,7 +409,7 @@
         <GENSTAMP>1003</GENSTAMP>
         <GENSTAMP>1003</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -423,17 +423,17 @@
       <INODEID>16390</INODEID>
       <INODEID>16390</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814891</MTIME>
-      <ATIME>1386003814891</ATIME>
+      <MTIME>1387010471547</MTIME>
+      <ATIME>1387010471547</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>40</RPC_CALLID>
       <RPC_CALLID>40</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -544,8 +544,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814914</MTIME>
-      <ATIME>1386003814891</ATIME>
+      <MTIME>1387010471588</MTIME>
+      <ATIME>1387010471547</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -565,7 +565,7 @@
         <GENSTAMP>1006</GENSTAMP>
         <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -579,17 +579,17 @@
       <INODEID>16391</INODEID>
       <INODEID>16391</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814916</MTIME>
-      <ATIME>1386003814916</ATIME>
+      <MTIME>1387010471595</MTIME>
+      <ATIME>1387010471595</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>52</RPC_CALLID>
       <RPC_CALLID>52</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -700,8 +700,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003814938</MTIME>
-      <ATIME>1386003814916</ATIME>
+      <MTIME>1387010471651</MTIME>
+      <ATIME>1387010471595</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -721,7 +721,7 @@
         <GENSTAMP>1009</GENSTAMP>
         <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -733,12 +733,12 @@
       <TXID>56</TXID>
       <TXID>56</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
       <TRG>/file_concat_target</TRG>
-      <TIMESTAMP>1386003814940</TIMESTAMP>
+      <TIMESTAMP>1387010471663</TIMESTAMP>
       <SOURCES>
       <SOURCES>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE2>/file_concat_1</SOURCE2>
         <SOURCE2>/file_concat_1</SOURCE2>
       </SOURCES>
       </SOURCES>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>63</RPC_CALLID>
       <RPC_CALLID>63</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -750,14 +750,14 @@
       <INODEID>16392</INODEID>
       <INODEID>16392</INODEID>
       <PATH>/file_symlink</PATH>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
       <VALUE>/file_concat_target</VALUE>
-      <MTIME>1386003814956</MTIME>
-      <ATIME>1386003814956</ATIME>
+      <MTIME>1387010471674</MTIME>
+      <ATIME>1387010471674</ATIME>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>511</MODE>
         <MODE>511</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>64</RPC_CALLID>
       <RPC_CALLID>64</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -768,14 +768,14 @@
       <DELEGATION_TOKEN_IDENTIFIER>
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
-        <OWNER>aagarwal</OWNER>
+        <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
         <REALUSER></REALUSER>
-        <ISSUE_DATE>1386003814961</ISSUE_DATE>
-        <MAX_DATE>1386608614961</MAX_DATE>
+        <ISSUE_DATE>1387010471682</ISSUE_DATE>
+        <MAX_DATE>1387615271682</MAX_DATE>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
       </DELEGATION_TOKEN_IDENTIFIER>
-      <EXPIRY_TIME>1386090214961</EXPIRY_TIME>
+      <EXPIRY_TIME>1387096871682</EXPIRY_TIME>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -785,14 +785,14 @@
       <DELEGATION_TOKEN_IDENTIFIER>
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
-        <OWNER>aagarwal</OWNER>
+        <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
         <REALUSER></REALUSER>
-        <ISSUE_DATE>1386003814961</ISSUE_DATE>
-        <MAX_DATE>1386608614961</MAX_DATE>
+        <ISSUE_DATE>1387010471682</ISSUE_DATE>
+        <MAX_DATE>1387615271682</MAX_DATE>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
       </DELEGATION_TOKEN_IDENTIFIER>
-      <EXPIRY_TIME>1386090215078</EXPIRY_TIME>
+      <EXPIRY_TIME>1387096871717</EXPIRY_TIME>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -802,11 +802,11 @@
       <DELEGATION_TOKEN_IDENTIFIER>
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
-        <OWNER>aagarwal</OWNER>
+        <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
         <REALUSER></REALUSER>
-        <ISSUE_DATE>1386003814961</ISSUE_DATE>
-        <MAX_DATE>1386608614961</MAX_DATE>
+        <ISSUE_DATE>1387010471682</ISSUE_DATE>
+        <MAX_DATE>1387615271682</MAX_DATE>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
       </DELEGATION_TOKEN_IDENTIFIER>
     </DATA>
     </DATA>
@@ -816,13 +816,11 @@
     <DATA>
     <DATA>
       <TXID>61</TXID>
       <TXID>61</TXID>
       <POOLNAME>poolparty</POOLNAME>
       <POOLNAME>poolparty</POOLNAME>
-      <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
-        <GROUPNAME>staff</GROUPNAME>
-        <MODE>493</MODE>
-      </PERMISSION_STATUS>
-      <WEIGHT>100</WEIGHT>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <OWNERNAME>andrew</OWNERNAME>
+      <GROUPNAME>andrew</GROUPNAME>
+      <MODE>493</MODE>
+      <LIMIT>9223372036854775807</LIMIT>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>68</RPC_CALLID>
       <RPC_CALLID>68</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -834,8 +832,8 @@
       <OWNERNAME>carlton</OWNERNAME>
       <OWNERNAME>carlton</OWNERNAME>
       <GROUPNAME>party</GROUPNAME>
       <GROUPNAME>party</GROUPNAME>
       <MODE>448</MODE>
       <MODE>448</MODE>
-      <WEIGHT>1989</WEIGHT>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <LIMIT>1989</LIMIT>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>69</RPC_CALLID>
       <RPC_CALLID>69</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -848,7 +846,7 @@
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <POOL>poolparty</POOL>
       <POOL>poolparty</POOL>
       <EXPIRATION>-1</EXPIRATION>
       <EXPIRATION>-1</EXPIRATION>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>70</RPC_CALLID>
       <RPC_CALLID>70</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -858,7 +856,7 @@
       <TXID>64</TXID>
       <TXID>64</TXID>
       <ID>1</ID>
       <ID>1</ID>
       <PATH>/bar2</PATH>
       <PATH>/bar2</PATH>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>71</RPC_CALLID>
       <RPC_CALLID>71</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -867,7 +865,7 @@
     <DATA>
     <DATA>
       <TXID>65</TXID>
       <TXID>65</TXID>
       <ID>1</ID>
       <ID>1</ID>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>72</RPC_CALLID>
       <RPC_CALLID>72</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -876,7 +874,7 @@
     <DATA>
     <DATA>
       <TXID>66</TXID>
       <TXID>66</TXID>
       <POOLNAME>poolparty</POOLNAME>
       <POOLNAME>poolparty</POOLNAME>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>73</RPC_CALLID>
       <RPC_CALLID>73</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -888,17 +886,17 @@
       <INODEID>16393</INODEID>
       <INODEID>16393</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003815135</MTIME>
-      <ATIME>1386003815135</ATIME>
+      <MTIME>1387010471802</MTIME>
+      <ATIME>1387010471802</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-1253204429_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>f583267a-ef8c-4f3f-9014-b067b83945ad</RPC_CLIENTID>
+      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
       <RPC_CALLID>74</RPC_CALLID>
       <RPC_CALLID>74</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -955,7 +953,7 @@
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
     <DATA>
       <TXID>73</TXID>
       <TXID>73</TXID>
-      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-1253204429_1</LEASEHOLDER>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-52011019_1</LEASEHOLDER>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
     </DATA>
     </DATA>
@@ -968,8 +966,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1386003817462</MTIME>
-      <ATIME>1386003815135</ATIME>
+      <MTIME>1387010474126</MTIME>
+      <ATIME>1387010471802</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -979,7 +977,7 @@
         <GENSTAMP>1011</GENSTAMP>
         <GENSTAMP>1011</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
-        <USERNAME>aagarwal</USERNAME>
+        <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>

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

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

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

@@ -71,6 +71,12 @@ Trunk (Unreleased)
     MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing.
     MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing.
     (Srikanth Sundarrajan via amareshwari)
     (Srikanth Sundarrajan via amareshwari)
 
 
+    MAPREDUCE-5197. Add a service for checkpointing task state.
+    (Carlo Curino via cdouglas)
+
+    MAPREDUCE-5189. Add policies and wiring to respond to preemption requests
+    from YARN. (Carlo Curino via cdouglas)
+
   BUG FIXES
   BUG FIXES
 
 
     MAPREDUCE-4272. SortedRanges.Range#compareTo is not spec compliant.
     MAPREDUCE-4272. SortedRanges.Range#compareTo is not spec compliant.
@@ -237,6 +243,15 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5656. bzip2 codec can drop records when reading data in splits
     MAPREDUCE-5656. bzip2 codec can drop records when reading data in splits
     (jlowe)
     (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
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -290,6 +305,9 @@ Release 2.3.0 - UNRELEASED
     event: TA_TOO_MANY_FETCH_FAILURE at KILLED for TaskAttemptImpl (Gera
     event: TA_TOO_MANY_FETCH_FAILURE at KILLED for TaskAttemptImpl (Gera
     Shegalov via jlowe)
     Shegalov via jlowe)
 
 
+    MAPREDUCE-5674. Missing start and finish time in mapred.JobStatus.
+    (Chuan Liu via cnauroth)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

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

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

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

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

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