Browse Source

HDFS-5511. improve CacheManipulator interface to allow better unit testing (cmccabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1543676 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 11 years ago
parent
commit
efea68dc35
14 changed files with 113 additions and 110 deletions
  1. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java
  2. 49 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
  3. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  4. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  5. 9 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  7. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MappableBlock.java
  9. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
  10. 9 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCachingStrategy.java
  11. 26 44
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
  12. 4 32
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java
  13. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java
  14. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java

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

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

+ 49 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java

@@ -98,9 +98,6 @@ public class NativeIO {
 
     private static final Log LOG = LogFactory.getLog(NativeIO.class);
 
-    @VisibleForTesting
-    public static CacheTracker cacheTracker = null;
-    
     private static boolean nativeLoaded = false;
     private static boolean fadvisePossible = true;
     private static boolean syncFileRangePossible = true;
@@ -111,18 +108,61 @@ public class NativeIO {
 
     private static long cacheTimeout = -1;
 
-    public static interface CacheTracker {
-      public void fadvise(String identifier, long offset, long len, int flags);
+    private static CacheManipulator cacheManipulator = new CacheManipulator();
+
+    public static CacheManipulator getCacheManipulator() {
+      return cacheManipulator;
     }
 
-    public static CacheManipulator cacheManipulator = new CacheManipulator();
+    public static void setCacheManipulator(CacheManipulator cacheManipulator) {
+      POSIX.cacheManipulator = cacheManipulator;
+    }
 
+    /**
+     * Used to manipulate the operating system cache.
+     */
     @VisibleForTesting
     public static class CacheManipulator {
       public void mlock(String identifier, ByteBuffer buffer,
           long len) throws IOException {
         POSIX.mlock(buffer, len);
       }
+
+      public long getMemlockLimit() {
+        return NativeIO.getMemlockLimit();
+      }
+
+      public long getOperatingSystemPageSize() {
+        return NativeIO.getOperatingSystemPageSize();
+      }
+
+      public void posixFadviseIfPossible(String identifier,
+        FileDescriptor fd, long offset, long len, int flags)
+            throws NativeIOException {
+        NativeIO.POSIX.posixFadviseIfPossible(identifier, fd, offset,
+            len, flags);
+      }
+    }
+
+    /**
+     * A CacheManipulator used for testing which does not actually call mlock.
+     * This allows many tests to be run even when the operating system does not
+     * allow mlock, or only allows limited mlocking.
+     */
+    @VisibleForTesting
+    public static class NoMlockCacheManipulator extends CacheManipulator {
+      public void mlock(String identifier, ByteBuffer buffer,
+          long len) throws IOException {
+        LOG.info("mlocking " + identifier);
+      }
+
+      public long getMemlockLimit() {
+        return 1125899906842624L;
+      }
+
+      public long getOperatingSystemPageSize() {
+        return 4096;
+      }
     }
 
     static {
@@ -207,12 +247,9 @@ public class NativeIO {
      *
      * @throws NativeIOException if there is an error with the syscall
      */
-    public static void posixFadviseIfPossible(String identifier,
+    static void posixFadviseIfPossible(String identifier,
         FileDescriptor fd, long offset, long len, int flags)
         throws NativeIOException {
-      if (cacheTracker != null) {
-        cacheTracker.fadvise(identifier, offset, len, flags);
-      }
       if (nativeLoaded && fadvisePossible) {
         try {
           posix_fadvise(fd, offset, len, flags);
@@ -566,7 +603,7 @@ public class NativeIO {
    *         Long.MAX_VALUE if there is no limit;
    *         The number of bytes that can be locked into memory otherwise.
    */
-  public static long getMemlockLimit() {
+  static long getMemlockLimit() {
     return isAvailable() ? getMemlockLimit0() : 0;
   }
 
@@ -575,7 +612,7 @@ public class NativeIO {
   /**
    * @return the operating system's page size.
    */
-  public static long getOperatingSystemPageSize() {
+  static long getOperatingSystemPageSize() {
     try {
       Field f = Unsafe.class.getDeclaredField("theUnsafe");
       f.setAccessible(true);

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

@@ -199,6 +199,9 @@ Trunk (Unreleased)
 
     HDFS-5366. recaching improvements (cmccabe)
 
+    HDFS-5511. improve CacheManipulator interface to allow better unit testing
+    (cmccabe)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 

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

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

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

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

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

@@ -667,7 +667,7 @@ public class DataNode extends Configured
             " size (%s) is greater than zero and native code is not available.",
             DFS_DATANODE_MAX_LOCKED_MEMORY_KEY));
       }
-      long ulimit = NativeIO.getMemlockLimit();
+      long ulimit = NativeIO.POSIX.getCacheManipulator().getMemlockLimit();
       if (dnConf.maxLockedMemory > ulimit) {
       throw new RuntimeException(String.format(
           "Cannot start datanode because the configured max locked memory" +

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

@@ -163,7 +163,8 @@ public class FsDatasetCache {
   private final UsedBytesCount usedBytesCount;
 
   public static class PageRounder {
-    private final long osPageSize = NativeIO.getOperatingSystemPageSize();
+    private final long osPageSize =
+        NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize();
 
     /**
      * Round up a number to the operating system page size.

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

@@ -82,7 +82,7 @@ public class MappableBlock implements Closeable {
         throw new IOException("Block InputStream has no FileChannel.");
       }
       mmap = blockChannel.map(MapMode.READ_ONLY, 0, length);
-      NativeIO.POSIX.cacheManipulator.mlock(blockFileName, mmap, length);
+      NativeIO.POSIX.getCacheManipulator().mlock(blockFileName, mmap, length);
       verifyChecksum(length, metaIn, blockChannel, blockFileName);
       mappableBlock = new MappableBlock(mmap, length);
     } finally {

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java

@@ -113,7 +113,8 @@ public class TestDatanodeConfig {
   @Test(timeout=60000)
   public void testMemlockLimit() throws Exception {
     assumeTrue(NativeIO.isAvailable());
-    final long memlockLimit = NativeIO.getMemlockLimit();
+    final long memlockLimit =
+        NativeIO.POSIX.getCacheManipulator().getMemlockLimit();
 
     // Can't increase the memlock limit past the maximum.
     assumeTrue(memlockLimit != Long.MAX_VALUE);

+ 9 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCachingStrategy.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
+import java.io.FileDescriptor;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Map;
@@ -36,7 +37,8 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
-import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheTracker;
+import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
+import org.apache.hadoop.io.nativeio.NativeIOException;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -54,7 +56,7 @@ public class TestCachingStrategy {
     EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
 
     // Track calls to posix_fadvise.
-    NativeIO.POSIX.cacheTracker = tracker;
+    NativeIO.POSIX.setCacheManipulator(tracker);
     
     // Normally, we wait for a few megabytes of data to be read or written 
     // before dropping the cache.  This is to avoid an excessive number of
@@ -106,12 +108,13 @@ public class TestCachingStrategy {
     }
   }
 
-  private static class TestRecordingCacheTracker implements CacheTracker {
+  private static class TestRecordingCacheTracker extends CacheManipulator {
     private final Map<String, Stats> map = new TreeMap<String, Stats>();
 
     @Override
-    synchronized public void fadvise(String name,
-        long offset, long len, int flags) {
+    public void posixFadviseIfPossible(String name,
+      FileDescriptor fd, long offset, long len, int flags)
+          throws NativeIOException {
       if ((len < 0) || (len > Integer.MAX_VALUE)) {
         throw new RuntimeException("invalid length of " + len +
             " passed to posixFadviseIfPossible");
@@ -126,6 +129,7 @@ public class TestCachingStrategy {
         map.put(name, stats);
       }
       stats.fadvise((int)offset, (int)len, flags);
+      super.posixFadviseIfPossible(name, fd, offset, len, flags);
     }
 
     synchronized void clear() {

+ 26 - 44
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java

@@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
+import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.MetricsAsserts;
@@ -99,7 +100,6 @@ public class TestFsDatasetCache {
   @Before
   public void setUp() throws Exception {
     assumeTrue(!Path.WINDOWS);
-    assumeTrue(NativeIO.getMemlockLimit() >= CACHE_CAPACITY);
     conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY, true);
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS,
@@ -122,18 +122,8 @@ public class TestFsDatasetCache {
 
     spyNN = DataNodeTestUtils.spyOnBposToNN(dn, nn);
 
-    prevCacheManipulator = NativeIO.POSIX.cacheManipulator;
-
-    // Save the current CacheManipulator and replace it at the end of the test
-    // Stub out mlock calls to avoid failing when not enough memory is lockable
-    // by the operating system.
-    NativeIO.POSIX.cacheManipulator = new CacheManipulator() {
-      @Override
-      public void mlock(String identifier,
-          ByteBuffer mmap, long length) throws IOException {
-        LOG.info("mlocking " + identifier);
-      }
-    };
+    prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
+    NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
   }
 
   @After
@@ -145,7 +135,7 @@ public class TestFsDatasetCache {
       cluster.shutdown();
     }
     // Restore the original CacheManipulator
-    NativeIO.POSIX.cacheManipulator = prevCacheManipulator;
+    NativeIO.POSIX.setCacheManipulator(prevCacheManipulator);
   }
 
   private static void setHeartbeatResponse(DatanodeCommand[] cmds)
@@ -222,7 +212,8 @@ public class TestFsDatasetCache {
           if (tries++ > 10) {
             LOG.info("verifyExpectedCacheUsage: expected " +
                 expected + ", got " + curDnCacheUsed + "; " +
-                "memlock limit = " + NativeIO.getMemlockLimit() +
+                "memlock limit = " +
+                NativeIO.POSIX.getCacheManipulator().getMemlockLimit() +
                 ".  Waiting...");
           }
           return false;
@@ -297,40 +288,31 @@ public class TestFsDatasetCache {
    */
   @Test(timeout=600000)
   public void testCacheAndUncacheBlockWithRetries() throws Exception {
-    CacheManipulator prevCacheManipulator = NativeIO.POSIX.cacheManipulator;
-    
-    try {
-      NativeIO.POSIX.cacheManipulator = new CacheManipulator() {
-        private final Set<String> seenIdentifiers = new HashSet<String>();
-        
-        @Override
-        public void mlock(String identifier,
-            ByteBuffer mmap, long length) throws IOException {
-          if (seenIdentifiers.contains(identifier)) {
-            // mlock succeeds the second time.
-            LOG.info("mlocking " + identifier);
-            return;
-          }
-          seenIdentifiers.add(identifier);
-          throw new IOException("injecting IOException during mlock of " +
-              identifier);
+    // We don't have to save the previous cacheManipulator
+    // because it will be reinstalled by the @After function.
+    NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator() {
+      private final Set<String> seenIdentifiers = new HashSet<String>();
+      
+      @Override
+      public void mlock(String identifier,
+          ByteBuffer mmap, long length) throws IOException {
+        if (seenIdentifiers.contains(identifier)) {
+          // mlock succeeds the second time.
+          LOG.info("mlocking " + identifier);
+          return;
         }
-      };
-      testCacheAndUncacheBlock();
-    } finally {
-      NativeIO.POSIX.cacheManipulator = prevCacheManipulator;
-    }
+        seenIdentifiers.add(identifier);
+        throw new IOException("injecting IOException during mlock of " +
+            identifier);
+      }
+    });
+    testCacheAndUncacheBlock();
   }
 
   @Test(timeout=600000)
   public void testFilesExceedMaxLockedMemory() throws Exception {
     LOG.info("beginning testFilesExceedMaxLockedMemory");
 
-    // We don't want to deal with page rounding issues, so skip this
-    // test if page size is weird
-    long osPageSize = NativeIO.getOperatingSystemPageSize();
-    assumeTrue(osPageSize == 4096); 
-
     // Create some test files that will exceed total cache capacity
     final int numFiles = 5;
     final long fileSize = 15000;
@@ -411,7 +393,7 @@ public class TestFsDatasetCache {
     assertEquals("Unexpected cache capacity", CACHE_CAPACITY, cacheCapacity);
     assertEquals("Unexpected amount of cache used", current, cacheUsed);
 
-    NativeIO.POSIX.cacheManipulator = new NativeIO.POSIX.CacheManipulator() {
+    NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator() {
       @Override
       public void mlock(String identifier,
           ByteBuffer mmap, long length) throws IOException {
@@ -422,7 +404,7 @@ public class TestFsDatasetCache {
           Assert.fail();
         }
       }
-    };
+    });
     // Starting caching each block in succession.  The usedBytes amount
     // should increase, even though caching doesn't complete on any of them.
     for (int i=0; i<NUM_BLOCKS; i++) {

+ 4 - 32
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java

@@ -61,12 +61,12 @@ import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
+import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.GSet;
 import org.junit.After;
-import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -98,18 +98,8 @@ public class TestPathBasedCacheRequests {
     cluster.waitActive();
     dfs = cluster.getFileSystem();
     proto = cluster.getNameNodeRpc();
-    prevCacheManipulator = NativeIO.POSIX.cacheManipulator;
-
-    // Save the current CacheManipulator and replace it at the end of the test
-    // Stub out mlock calls to avoid failing when not enough memory is lockable
-    // by the operating system.
-    NativeIO.POSIX.cacheManipulator = new CacheManipulator() {
-      @Override
-      public void mlock(String identifier,
-          ByteBuffer mmap, long length) throws IOException {
-        LOG.info("mlocking " + identifier);
-      }
-    };
+    prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
+    NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
   }
 
   @After
@@ -118,7 +108,7 @@ public class TestPathBasedCacheRequests {
       cluster.shutdown();
     }
     // Restore the original CacheManipulator
-    NativeIO.POSIX.cacheManipulator = prevCacheManipulator;
+    NativeIO.POSIX.setCacheManipulator(prevCacheManipulator);
   }
 
   @Test(timeout=60000)
@@ -654,20 +644,6 @@ public class TestPathBasedCacheRequests {
   // Most Linux installs will allow non-root users to lock 64KB.
   private static final long CACHE_CAPACITY = 64 * 1024 / NUM_DATANODES;
 
-  /**
-   * Return true if we can test DN caching.
-   */
-  private static boolean canTestDatanodeCaching() {
-    if (!NativeIO.isAvailable()) {
-      // Need NativeIO in order to cache blocks on the DN.
-      return false;
-    }
-    if (NativeIO.getMemlockLimit() < CACHE_CAPACITY) {
-      return false;
-    }
-    return true;
-  }
-
   private static HdfsConfiguration createCachingConf() {
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -681,7 +657,6 @@ public class TestPathBasedCacheRequests {
 
   @Test(timeout=120000)
   public void testWaitForCachedReplicas() throws Exception {
-    Assume.assumeTrue(canTestDatanodeCaching());
     HdfsConfiguration conf = createCachingConf();
     FileSystemTestHelper helper = new FileSystemTestHelper();
     MiniDFSCluster cluster =
@@ -739,7 +714,6 @@ public class TestPathBasedCacheRequests {
   @Test(timeout=120000)
   public void testAddingPathBasedCacheDirectivesWhenCachingIsDisabled()
       throws Exception {
-    Assume.assumeTrue(canTestDatanodeCaching());
     HdfsConfiguration conf = createCachingConf();
     conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, false);
     MiniDFSCluster cluster =
@@ -787,7 +761,6 @@ public class TestPathBasedCacheRequests {
 
   @Test(timeout=120000)
   public void testWaitForCachedReplicasInDirectory() throws Exception {
-    Assume.assumeTrue(canTestDatanodeCaching());
     HdfsConfiguration conf = createCachingConf();
     MiniDFSCluster cluster =
       new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
@@ -839,7 +812,6 @@ public class TestPathBasedCacheRequests {
    */
   @Test(timeout=120000)
   public void testReplicationFactor() throws Exception {
-    Assume.assumeTrue(canTestDatanodeCaching());
     HdfsConfiguration conf = createCachingConf();
     MiniDFSCluster cluster =
       new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java

@@ -69,7 +69,7 @@ public class FadvisedChunkedFile extends ChunkedFile {
     }
     if (manageOsCache && getEndOffset() - getStartOffset() > 0) {
       try {
-        NativeIO.POSIX.posixFadviseIfPossible(identifier,
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
             fd,
             getStartOffset(), getEndOffset() - getStartOffset(),
             NativeIO.POSIX.POSIX_FADV_DONTNEED);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java

@@ -79,7 +79,7 @@ public class FadvisedFileRegion extends DefaultFileRegion {
   public void transferSuccessful() {
     if (manageOsCache && getCount() > 0) {
       try {
-        NativeIO.POSIX.posixFadviseIfPossible(identifier,
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
            fd, getPosition(), getCount(),
            NativeIO.POSIX.POSIX_FADV_DONTNEED);
       } catch (Throwable t) {