|
@@ -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));
|
|
}
|
|
}
|
|
}
|
|
}
|