|
@@ -31,7 +31,6 @@ import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
-import java.nio.ByteBuffer;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Iterator;
|
|
@@ -54,8 +53,10 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
|
|
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
|
|
|
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
|
|
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
|
|
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
|
|
@@ -76,8 +77,8 @@ import org.junit.Test;
|
|
|
|
|
|
import com.google.common.base.Supplier;
|
|
|
|
|
|
-public class TestPathBasedCacheRequests {
|
|
|
- static final Log LOG = LogFactory.getLog(TestPathBasedCacheRequests.class);
|
|
|
+public class TestCacheDirectives {
|
|
|
+ static final Log LOG = LogFactory.getLog(TestCacheDirectives.class);
|
|
|
|
|
|
private static final UserGroupInformation unprivilegedUser =
|
|
|
UserGroupInformation.createRemoteUser("unprivilegedUser");
|
|
@@ -312,24 +313,25 @@ public class TestPathBasedCacheRequests {
|
|
|
}
|
|
|
|
|
|
private static void validateListAll(
|
|
|
- RemoteIterator<PathBasedCacheDirective> iter,
|
|
|
+ RemoteIterator<CacheDirectiveEntry> iter,
|
|
|
Long... ids) throws Exception {
|
|
|
for (Long id: ids) {
|
|
|
assertTrue("Unexpectedly few elements", iter.hasNext());
|
|
|
- assertEquals("Unexpected directive ID", id, iter.next().getId());
|
|
|
+ assertEquals("Unexpected directive ID", id,
|
|
|
+ iter.next().getInfo().getId());
|
|
|
}
|
|
|
assertFalse("Unexpectedly many list elements", iter.hasNext());
|
|
|
}
|
|
|
|
|
|
private static long addAsUnprivileged(
|
|
|
- final PathBasedCacheDirective directive) throws Exception {
|
|
|
+ final CacheDirectiveInfo directive) throws Exception {
|
|
|
return unprivilegedUser
|
|
|
.doAs(new PrivilegedExceptionAction<Long>() {
|
|
|
@Override
|
|
|
public Long run() throws IOException {
|
|
|
DistributedFileSystem myDfs =
|
|
|
(DistributedFileSystem) FileSystem.get(conf);
|
|
|
- return myDfs.addPathBasedCacheDirective(directive);
|
|
|
+ return myDfs.addCacheDirective(directive);
|
|
|
}
|
|
|
});
|
|
|
}
|
|
@@ -345,15 +347,15 @@ public class TestPathBasedCacheRequests {
|
|
|
proto.addCachePool(new CachePoolInfo("pool4").
|
|
|
setMode(new FsPermission((short)0)));
|
|
|
|
|
|
- PathBasedCacheDirective alpha = new PathBasedCacheDirective.Builder().
|
|
|
+ CacheDirectiveInfo alpha = new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/alpha")).
|
|
|
setPool("pool1").
|
|
|
build();
|
|
|
- PathBasedCacheDirective beta = new PathBasedCacheDirective.Builder().
|
|
|
+ CacheDirectiveInfo beta = new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/beta")).
|
|
|
setPool("pool2").
|
|
|
build();
|
|
|
- PathBasedCacheDirective delta = new PathBasedCacheDirective.Builder().
|
|
|
+ CacheDirectiveInfo delta = new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/delta")).
|
|
|
setPool("pool1").
|
|
|
build();
|
|
@@ -361,12 +363,12 @@ public class TestPathBasedCacheRequests {
|
|
|
long alphaId = addAsUnprivileged(alpha);
|
|
|
long alphaId2 = addAsUnprivileged(alpha);
|
|
|
assertFalse("Expected to get unique directives when re-adding an "
|
|
|
- + "existing PathBasedCacheDirective",
|
|
|
+ + "existing CacheDirectiveInfo",
|
|
|
alphaId == alphaId2);
|
|
|
long betaId = addAsUnprivileged(beta);
|
|
|
|
|
|
try {
|
|
|
- addAsUnprivileged(new PathBasedCacheDirective.Builder().
|
|
|
+ addAsUnprivileged(new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/unicorn")).
|
|
|
setPool("no_such_pool").
|
|
|
build());
|
|
@@ -376,7 +378,7 @@ public class TestPathBasedCacheRequests {
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- addAsUnprivileged(new PathBasedCacheDirective.Builder().
|
|
|
+ addAsUnprivileged(new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/blackhole")).
|
|
|
setPool("pool4").
|
|
|
build());
|
|
@@ -388,7 +390,7 @@ public class TestPathBasedCacheRequests {
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- addAsUnprivileged(new PathBasedCacheDirective.Builder().
|
|
|
+ addAsUnprivileged(new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/illegal:path/")).
|
|
|
setPool("pool1").
|
|
|
build());
|
|
@@ -399,12 +401,12 @@ public class TestPathBasedCacheRequests {
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- addAsUnprivileged(new PathBasedCacheDirective.Builder().
|
|
|
+ addAsUnprivileged(new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/emptypoolname")).
|
|
|
setReplication((short)1).
|
|
|
setPool("").
|
|
|
build());
|
|
|
- fail("expected an error when adding a PathBasedCache " +
|
|
|
+ fail("expected an error when adding a cache " +
|
|
|
"directive with an empty pool name.");
|
|
|
} catch (InvalidRequestException e) {
|
|
|
GenericTestUtils.assertExceptionContains("Invalid empty pool name", e);
|
|
@@ -415,75 +417,75 @@ public class TestPathBasedCacheRequests {
|
|
|
// We expect the following to succeed, because DistributedFileSystem
|
|
|
// qualifies the path.
|
|
|
long relativeId = addAsUnprivileged(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("relative")).
|
|
|
setPool("pool1").
|
|
|
build());
|
|
|
|
|
|
- RemoteIterator<PathBasedCacheDirective> iter;
|
|
|
- iter = dfs.listPathBasedCacheDirectives(null);
|
|
|
+ RemoteIterator<CacheDirectiveEntry> iter;
|
|
|
+ iter = dfs.listCacheDirectives(null);
|
|
|
validateListAll(iter, alphaId, alphaId2, betaId, deltaId, relativeId );
|
|
|
- iter = dfs.listPathBasedCacheDirectives(
|
|
|
- new PathBasedCacheDirective.Builder().setPool("pool3").build());
|
|
|
+ iter = dfs.listCacheDirectives(
|
|
|
+ new CacheDirectiveInfo.Builder().setPool("pool3").build());
|
|
|
assertFalse(iter.hasNext());
|
|
|
- iter = dfs.listPathBasedCacheDirectives(
|
|
|
- new PathBasedCacheDirective.Builder().setPool("pool1").build());
|
|
|
+ iter = dfs.listCacheDirectives(
|
|
|
+ new CacheDirectiveInfo.Builder().setPool("pool1").build());
|
|
|
validateListAll(iter, alphaId, alphaId2, deltaId, relativeId );
|
|
|
- iter = dfs.listPathBasedCacheDirectives(
|
|
|
- new PathBasedCacheDirective.Builder().setPool("pool2").build());
|
|
|
+ iter = dfs.listCacheDirectives(
|
|
|
+ new CacheDirectiveInfo.Builder().setPool("pool2").build());
|
|
|
validateListAll(iter, betaId);
|
|
|
|
|
|
- dfs.removePathBasedCacheDirective(betaId);
|
|
|
- iter = dfs.listPathBasedCacheDirectives(
|
|
|
- new PathBasedCacheDirective.Builder().setPool("pool2").build());
|
|
|
+ dfs.removeCacheDirective(betaId);
|
|
|
+ iter = dfs.listCacheDirectives(
|
|
|
+ new CacheDirectiveInfo.Builder().setPool("pool2").build());
|
|
|
assertFalse(iter.hasNext());
|
|
|
|
|
|
try {
|
|
|
- dfs.removePathBasedCacheDirective(betaId);
|
|
|
+ dfs.removeCacheDirective(betaId);
|
|
|
fail("expected an error when removing a non-existent ID");
|
|
|
} catch (InvalidRequestException e) {
|
|
|
GenericTestUtils.assertExceptionContains("No directive with ID", e);
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- proto.removePathBasedCacheDirective(-42l);
|
|
|
+ proto.removeCacheDirective(-42l);
|
|
|
fail("expected an error when removing a negative ID");
|
|
|
} catch (InvalidRequestException e) {
|
|
|
GenericTestUtils.assertExceptionContains(
|
|
|
"Invalid negative ID", e);
|
|
|
}
|
|
|
try {
|
|
|
- proto.removePathBasedCacheDirective(43l);
|
|
|
+ proto.removeCacheDirective(43l);
|
|
|
fail("expected an error when removing a non-existent ID");
|
|
|
} catch (InvalidRequestException e) {
|
|
|
GenericTestUtils.assertExceptionContains("No directive with ID", e);
|
|
|
}
|
|
|
|
|
|
- dfs.removePathBasedCacheDirective(alphaId);
|
|
|
- dfs.removePathBasedCacheDirective(alphaId2);
|
|
|
- dfs.removePathBasedCacheDirective(deltaId);
|
|
|
+ dfs.removeCacheDirective(alphaId);
|
|
|
+ dfs.removeCacheDirective(alphaId2);
|
|
|
+ dfs.removeCacheDirective(deltaId);
|
|
|
|
|
|
- dfs.modifyPathBasedCacheDirective(new PathBasedCacheDirective.Builder().
|
|
|
+ dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder().
|
|
|
setId(relativeId).
|
|
|
setReplication((short)555).
|
|
|
build());
|
|
|
- iter = dfs.listPathBasedCacheDirectives(null);
|
|
|
+ iter = dfs.listCacheDirectives(null);
|
|
|
assertTrue(iter.hasNext());
|
|
|
- PathBasedCacheDirective modified = iter.next();
|
|
|
+ CacheDirectiveInfo modified = iter.next().getInfo();
|
|
|
assertEquals(relativeId, modified.getId().longValue());
|
|
|
assertEquals((short)555, modified.getReplication().shortValue());
|
|
|
- dfs.removePathBasedCacheDirective(relativeId);
|
|
|
- iter = dfs.listPathBasedCacheDirectives(null);
|
|
|
+ dfs.removeCacheDirective(relativeId);
|
|
|
+ iter = dfs.listCacheDirectives(null);
|
|
|
assertFalse(iter.hasNext());
|
|
|
|
|
|
// Verify that PBCDs with path "." work correctly
|
|
|
- PathBasedCacheDirective directive =
|
|
|
- new PathBasedCacheDirective.Builder().setPath(new Path("."))
|
|
|
+ CacheDirectiveInfo directive =
|
|
|
+ new CacheDirectiveInfo.Builder().setPath(new Path("."))
|
|
|
.setPool("pool1").build();
|
|
|
- long id = dfs.addPathBasedCacheDirective(directive);
|
|
|
- dfs.modifyPathBasedCacheDirective(new PathBasedCacheDirective.Builder(
|
|
|
+ long id = dfs.addCacheDirective(directive);
|
|
|
+ dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(
|
|
|
directive).setId(id).setReplication((short)2).build());
|
|
|
- dfs.removePathBasedCacheDirective(id);
|
|
|
+ dfs.removeCacheDirective(id);
|
|
|
}
|
|
|
|
|
|
@Test(timeout=60000)
|
|
@@ -519,15 +521,15 @@ public class TestPathBasedCacheRequests {
|
|
|
String entryPrefix = "/party-";
|
|
|
long prevId = -1;
|
|
|
for (int i=0; i<numEntries; i++) {
|
|
|
- prevId = dfs.addPathBasedCacheDirective(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ prevId = dfs.addCacheDirective(
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path(entryPrefix + i)).setPool(pool).build());
|
|
|
}
|
|
|
- RemoteIterator<PathBasedCacheDirective> dit
|
|
|
- = dfs.listPathBasedCacheDirectives(null);
|
|
|
+ RemoteIterator<CacheDirectiveEntry> dit
|
|
|
+ = dfs.listCacheDirectives(null);
|
|
|
for (int i=0; i<numEntries; i++) {
|
|
|
assertTrue("Unexpected # of cache entries: " + i, dit.hasNext());
|
|
|
- PathBasedCacheDirective cd = dit.next();
|
|
|
+ CacheDirectiveInfo cd = dit.next().getInfo();
|
|
|
assertEquals(i+1, cd.getId().longValue());
|
|
|
assertEquals(entryPrefix + i, cd.getPath().toUri().getPath());
|
|
|
assertEquals(pool, cd.getPool());
|
|
@@ -548,18 +550,18 @@ public class TestPathBasedCacheRequests {
|
|
|
assertEquals(weight, (int)info.getWeight());
|
|
|
assertFalse("Unexpected # of cache pools found", pit.hasNext());
|
|
|
|
|
|
- dit = dfs.listPathBasedCacheDirectives(null);
|
|
|
+ dit = dfs.listCacheDirectives(null);
|
|
|
for (int i=0; i<numEntries; i++) {
|
|
|
assertTrue("Unexpected # of cache entries: " + i, dit.hasNext());
|
|
|
- PathBasedCacheDirective cd = dit.next();
|
|
|
+ CacheDirectiveInfo cd = dit.next().getInfo();
|
|
|
assertEquals(i+1, cd.getId().longValue());
|
|
|
assertEquals(entryPrefix + i, cd.getPath().toUri().getPath());
|
|
|
assertEquals(pool, cd.getPool());
|
|
|
}
|
|
|
assertFalse("Unexpected # of cache directives found", dit.hasNext());
|
|
|
|
|
|
- long nextId = dfs.addPathBasedCacheDirective(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ long nextId = dfs.addCacheDirective(
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/foobar")).setPool(pool).build());
|
|
|
assertEquals(prevId + 1, nextId);
|
|
|
}
|
|
@@ -691,22 +693,22 @@ public class TestPathBasedCacheRequests {
|
|
|
// Cache and check each path in sequence
|
|
|
int expected = 0;
|
|
|
for (int i=0; i<numFiles; i++) {
|
|
|
- PathBasedCacheDirective directive =
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ CacheDirectiveInfo directive =
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path(paths.get(i))).
|
|
|
setPool(pool).
|
|
|
build();
|
|
|
- nnRpc.addPathBasedCacheDirective(directive);
|
|
|
+ nnRpc.addCacheDirective(directive);
|
|
|
expected += numBlocksPerFile;
|
|
|
waitForCachedBlocks(namenode, expected, expected,
|
|
|
"testWaitForCachedReplicas:1");
|
|
|
}
|
|
|
// Uncache and check each path in sequence
|
|
|
- RemoteIterator<PathBasedCacheDirective> entries =
|
|
|
- nnRpc.listPathBasedCacheDirectives(0, null);
|
|
|
+ RemoteIterator<CacheDirectiveEntry> entries =
|
|
|
+ nnRpc.listCacheDirectives(0, null);
|
|
|
for (int i=0; i<numFiles; i++) {
|
|
|
- PathBasedCacheDirective directive = entries.next();
|
|
|
- nnRpc.removePathBasedCacheDirective(directive.getId());
|
|
|
+ CacheDirectiveEntry entry = entries.next();
|
|
|
+ nnRpc.removeCacheDirective(entry.getInfo().getId());
|
|
|
expected -= numBlocksPerFile;
|
|
|
waitForCachedBlocks(namenode, expected, expected,
|
|
|
"testWaitForCachedReplicas:2");
|
|
@@ -717,7 +719,7 @@ public class TestPathBasedCacheRequests {
|
|
|
}
|
|
|
|
|
|
@Test(timeout=120000)
|
|
|
- public void testAddingPathBasedCacheDirectivesWhenCachingIsDisabled()
|
|
|
+ public void testAddingCacheDirectiveInfosWhenCachingIsDisabled()
|
|
|
throws Exception {
|
|
|
HdfsConfiguration conf = createCachingConf();
|
|
|
conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, false);
|
|
@@ -743,22 +745,22 @@ public class TestPathBasedCacheRequests {
|
|
|
}
|
|
|
// Check the initial statistics at the namenode
|
|
|
waitForCachedBlocks(namenode, 0, 0,
|
|
|
- "testAddingPathBasedCacheDirectivesWhenCachingIsDisabled:0");
|
|
|
+ "testAddingCacheDirectiveInfosWhenCachingIsDisabled:0");
|
|
|
// Cache and check each path in sequence
|
|
|
int expected = 0;
|
|
|
for (int i=0; i<numFiles; i++) {
|
|
|
- PathBasedCacheDirective directive =
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ CacheDirectiveInfo directive =
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path(paths.get(i))).
|
|
|
setPool(pool).
|
|
|
build();
|
|
|
- dfs.addPathBasedCacheDirective(directive);
|
|
|
+ dfs.addCacheDirective(directive);
|
|
|
waitForCachedBlocks(namenode, expected, 0,
|
|
|
- "testAddingPathBasedCacheDirectivesWhenCachingIsDisabled:1");
|
|
|
+ "testAddingCacheDirectiveInfosWhenCachingIsDisabled:1");
|
|
|
}
|
|
|
Thread.sleep(20000);
|
|
|
waitForCachedBlocks(namenode, expected, 0,
|
|
|
- "testAddingPathBasedCacheDirectivesWhenCachingIsDisabled:2");
|
|
|
+ "testAddingCacheDirectiveInfosWhenCachingIsDisabled:2");
|
|
|
} finally {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
@@ -793,8 +795,8 @@ public class TestPathBasedCacheRequests {
|
|
|
waitForCachedBlocks(namenode, 0, 0,
|
|
|
"testWaitForCachedReplicasInDirectory:0");
|
|
|
// cache entire directory
|
|
|
- long id = dfs.addPathBasedCacheDirective(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ long id = dfs.addCacheDirective(
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/foo")).
|
|
|
setReplication((short)2).
|
|
|
setPool(pool).
|
|
@@ -802,22 +804,23 @@ public class TestPathBasedCacheRequests {
|
|
|
waitForCachedBlocks(namenode, 4, 8,
|
|
|
"testWaitForCachedReplicasInDirectory:1");
|
|
|
// Verify that listDirectives gives the stats we want.
|
|
|
- RemoteIterator<PathBasedCacheDirective> iter =
|
|
|
- dfs.listPathBasedCacheDirectives(new PathBasedCacheDirective.Builder().
|
|
|
+ RemoteIterator<CacheDirectiveEntry> iter =
|
|
|
+ dfs.listCacheDirectives(new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/foo")).
|
|
|
build());
|
|
|
- PathBasedCacheDirective directive = iter.next();
|
|
|
+ CacheDirectiveEntry entry = iter.next();
|
|
|
+ CacheDirectiveStats stats = entry.getStats();
|
|
|
Assert.assertEquals(Long.valueOf(2),
|
|
|
- directive.getFilesAffected());
|
|
|
+ stats.getFilesAffected());
|
|
|
Assert.assertEquals(Long.valueOf(
|
|
|
2 * numBlocksPerFile * BLOCK_SIZE * 2),
|
|
|
- directive.getBytesNeeded());
|
|
|
+ stats.getBytesNeeded());
|
|
|
Assert.assertEquals(Long.valueOf(
|
|
|
2 * numBlocksPerFile * BLOCK_SIZE * 2),
|
|
|
- directive.getBytesCached());
|
|
|
+ stats.getBytesCached());
|
|
|
|
|
|
- long id2 = dfs.addPathBasedCacheDirective(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ long id2 = dfs.addCacheDirective(
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/foo/bar")).
|
|
|
setReplication((short)4).
|
|
|
setPool(pool).
|
|
@@ -826,38 +829,40 @@ public class TestPathBasedCacheRequests {
|
|
|
waitForCachedBlocks(namenode, 4, 10,
|
|
|
"testWaitForCachedReplicasInDirectory:2");
|
|
|
// the directory directive's stats are unchanged
|
|
|
- iter = dfs.listPathBasedCacheDirectives(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ iter = dfs.listCacheDirectives(
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/foo")).
|
|
|
build());
|
|
|
- directive = iter.next();
|
|
|
+ entry = iter.next();
|
|
|
+ stats = entry.getStats();
|
|
|
Assert.assertEquals(Long.valueOf(2),
|
|
|
- directive.getFilesAffected());
|
|
|
+ stats.getFilesAffected());
|
|
|
Assert.assertEquals(Long.valueOf(
|
|
|
2 * numBlocksPerFile * BLOCK_SIZE * 2),
|
|
|
- directive.getBytesNeeded());
|
|
|
+ stats.getBytesNeeded());
|
|
|
Assert.assertEquals(Long.valueOf(
|
|
|
2 * numBlocksPerFile * BLOCK_SIZE * 2),
|
|
|
- directive.getBytesCached());
|
|
|
+ stats.getBytesCached());
|
|
|
// verify /foo/bar's stats
|
|
|
- iter = dfs.listPathBasedCacheDirectives(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ iter = dfs.listCacheDirectives(
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/foo/bar")).
|
|
|
build());
|
|
|
- directive = iter.next();
|
|
|
+ entry = iter.next();
|
|
|
+ stats = entry.getStats();
|
|
|
Assert.assertEquals(Long.valueOf(1),
|
|
|
- directive.getFilesAffected());
|
|
|
+ stats.getFilesAffected());
|
|
|
Assert.assertEquals(Long.valueOf(
|
|
|
4 * numBlocksPerFile * BLOCK_SIZE),
|
|
|
- directive.getBytesNeeded());
|
|
|
+ stats.getBytesNeeded());
|
|
|
// only 3 because the file only has 3 replicas, not 4 as requested.
|
|
|
Assert.assertEquals(Long.valueOf(
|
|
|
3 * numBlocksPerFile * BLOCK_SIZE),
|
|
|
- directive.getBytesCached());
|
|
|
+ stats.getBytesCached());
|
|
|
|
|
|
// remove and watch numCached go to 0
|
|
|
- dfs.removePathBasedCacheDirective(id);
|
|
|
- dfs.removePathBasedCacheDirective(id2);
|
|
|
+ dfs.removeCacheDirective(id);
|
|
|
+ dfs.removeCacheDirective(id2);
|
|
|
waitForCachedBlocks(namenode, 0, 0,
|
|
|
"testWaitForCachedReplicasInDirectory:3");
|
|
|
} finally {
|
|
@@ -899,8 +904,8 @@ public class TestPathBasedCacheRequests {
|
|
|
waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:0");
|
|
|
checkNumCachedReplicas(dfs, paths, 0, 0);
|
|
|
// cache directory
|
|
|
- long id = dfs.addPathBasedCacheDirective(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ long id = dfs.addCacheDirective(
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setPath(new Path("/foo")).
|
|
|
setReplication((short)1).
|
|
|
setPool(pool).
|
|
@@ -909,8 +914,8 @@ public class TestPathBasedCacheRequests {
|
|
|
checkNumCachedReplicas(dfs, paths, 4, 4);
|
|
|
// step up the replication factor
|
|
|
for (int i=2; i<=3; i++) {
|
|
|
- dfs.modifyPathBasedCacheDirective(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ dfs.modifyCacheDirective(
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setId(id).
|
|
|
setReplication((short)i).
|
|
|
build());
|
|
@@ -919,8 +924,8 @@ public class TestPathBasedCacheRequests {
|
|
|
}
|
|
|
// step it down
|
|
|
for (int i=2; i>=1; i--) {
|
|
|
- dfs.modifyPathBasedCacheDirective(
|
|
|
- new PathBasedCacheDirective.Builder().
|
|
|
+ dfs.modifyCacheDirective(
|
|
|
+ new CacheDirectiveInfo.Builder().
|
|
|
setId(id).
|
|
|
setReplication((short)i).
|
|
|
build());
|
|
@@ -928,7 +933,7 @@ public class TestPathBasedCacheRequests {
|
|
|
checkNumCachedReplicas(dfs, paths, 4, 4*i);
|
|
|
}
|
|
|
// remove and watch numCached go to 0
|
|
|
- dfs.removePathBasedCacheDirective(id);
|
|
|
+ dfs.removeCacheDirective(id);
|
|
|
waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:4");
|
|
|
checkNumCachedReplicas(dfs, paths, 0, 0);
|
|
|
} finally {
|