|
@@ -44,9 +44,13 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.balancer.Dispatcher;
|
|
|
import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
|
|
|
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.log4j.Level;
|
|
@@ -66,6 +70,8 @@ public class TestStorageMover {
|
|
|
).getLogger().setLevel(Level.ALL);
|
|
|
((Log4JLogger)LogFactory.getLog(Dispatcher.class)
|
|
|
).getLogger().setLevel(Level.ALL);
|
|
|
+ ((Log4JLogger)LogFactory.getLog(DataTransferProtocol.class)).getLogger()
|
|
|
+ .setLevel(Level.ALL);
|
|
|
}
|
|
|
|
|
|
private static final int BLOCK_SIZE = 1024;
|
|
@@ -80,6 +86,8 @@ public class TestStorageMover {
|
|
|
static {
|
|
|
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
|
|
|
+ DEFAULT_CONF.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
|
|
|
+ 2L);
|
|
|
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY, 2000L);
|
|
|
|
|
|
DEFAULT_POLICIES = BlockStoragePolicy.readBlockStorageSuite(DEFAULT_CONF);
|
|
@@ -87,6 +95,9 @@ public class TestStorageMover {
|
|
|
WARM = DEFAULT_POLICIES.getPolicy("WARM");
|
|
|
COLD = DEFAULT_POLICIES.getPolicy("COLD");
|
|
|
Dispatcher.setBlockMoveWaitTime(1000L);
|
|
|
+ Dispatcher.setDelayAfterErrors(1000L);
|
|
|
+ // do not create id file since we will eat up all the disk space
|
|
|
+ NameNodeConnector.setCreateIdFile(false);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -151,7 +162,7 @@ public class TestStorageMover {
|
|
|
|
|
|
ClusterScheme() {
|
|
|
this(DEFAULT_CONF, NUM_DATANODES, REPL,
|
|
|
- genStorageTypes(NUM_DATANODES, 1, 1), null);
|
|
|
+ genStorageTypes(NUM_DATANODES), null);
|
|
|
}
|
|
|
|
|
|
ClusterScheme(Configuration conf, int numDataNodes, short repl,
|
|
@@ -195,7 +206,7 @@ public class TestStorageMover {
|
|
|
dfs = cluster.getFileSystem();
|
|
|
}
|
|
|
|
|
|
- private void runBasicTest(boolean shotdown) throws Exception {
|
|
|
+ private void runBasicTest(boolean shutdown) throws Exception {
|
|
|
setupCluster();
|
|
|
try {
|
|
|
prepareNamespace();
|
|
@@ -205,7 +216,7 @@ public class TestStorageMover {
|
|
|
migrate();
|
|
|
verify(true);
|
|
|
} finally {
|
|
|
- if (shotdown) {
|
|
|
+ if (shutdown) {
|
|
|
shutdownCluster();
|
|
|
}
|
|
|
}
|
|
@@ -233,7 +244,7 @@ public class TestStorageMover {
|
|
|
/**
|
|
|
* Run the migration tool.
|
|
|
*/
|
|
|
- void migrate(String... args) throws Exception {
|
|
|
+ void migrate() throws Exception {
|
|
|
runMover();
|
|
|
Thread.sleep(5000); // let the NN finish deletion
|
|
|
}
|
|
@@ -242,6 +253,9 @@ public class TestStorageMover {
|
|
|
* Verify block locations after running the migration tool.
|
|
|
*/
|
|
|
void verify(boolean verifyAll) throws Exception {
|
|
|
+ for (DataNode dn : cluster.getDataNodes()) {
|
|
|
+ DataNodeTestUtils.triggerBlockReport(dn);
|
|
|
+ }
|
|
|
if (verifyAll) {
|
|
|
verifyNamespace();
|
|
|
} else {
|
|
@@ -308,7 +322,8 @@ public class TestStorageMover {
|
|
|
final Mover.StorageTypeDiff diff = new Mover.StorageTypeDiff(types,
|
|
|
lb.getStorageTypes());
|
|
|
Assert.assertTrue(fileStatus.getFullName(parent.toString())
|
|
|
- + " with policy " + policy + " has non-empty overlap: " + diff,
|
|
|
+ + " with policy " + policy + " has non-empty overlap: " + diff
|
|
|
+ + ", the corresponding block is " + lb.getBlock().getLocalBlock(),
|
|
|
diff.removeOverlap());
|
|
|
}
|
|
|
}
|
|
@@ -378,6 +393,7 @@ public class TestStorageMover {
|
|
|
return "[disk=" + disk + ", archive=" + archive + "]";
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
private static StorageType[][] genStorageTypes(int numDataNodes) {
|
|
|
return genStorageTypes(numDataNodes, 0, 0);
|
|
|
}
|
|
@@ -414,21 +430,6 @@ public class TestStorageMover {
|
|
|
return capacities;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * A normal case for Mover: move a file into archival storage
|
|
|
- */
|
|
|
- @Test
|
|
|
- public void testMigrateFileToArchival() throws Exception {
|
|
|
- final Path foo = new Path("/foo");
|
|
|
- Map<Path, BlockStoragePolicy> policyMap = Maps.newHashMap();
|
|
|
- policyMap.put(foo, COLD);
|
|
|
- NamespaceScheme nsScheme = new NamespaceScheme(null, Arrays.asList(foo),
|
|
|
- 2*BLOCK_SIZE, null, policyMap);
|
|
|
- ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
|
|
|
- NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES), null);
|
|
|
- new MigrationTest(clusterScheme, nsScheme).runBasicTest(true);
|
|
|
- }
|
|
|
-
|
|
|
private static class PathPolicyMap {
|
|
|
final Map<Path, BlockStoragePolicy> map = Maps.newHashMap();
|
|
|
final Path hot = new Path("/hot");
|
|
@@ -447,13 +448,13 @@ public class TestStorageMover {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
NamespaceScheme newNamespaceScheme() {
|
|
|
return new NamespaceScheme(Arrays.asList(hot, warm, cold),
|
|
|
files, BLOCK_SIZE/2, null, map);
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
+
|
|
|
+ /**
|
|
|
* Move hot files to warm and cold, warm files to hot and cold,
|
|
|
* and cold files to hot and warm.
|
|
|
*/
|
|
@@ -472,22 +473,42 @@ public class TestStorageMover {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * A normal case for Mover: move a file into archival storage
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testMigrateFileToArchival() throws Exception {
|
|
|
+ LOG.info("testMigrateFileToArchival");
|
|
|
+ final Path foo = new Path("/foo");
|
|
|
+ Map<Path, BlockStoragePolicy> policyMap = Maps.newHashMap();
|
|
|
+ policyMap.put(foo, COLD);
|
|
|
+ NamespaceScheme nsScheme = new NamespaceScheme(null, Arrays.asList(foo),
|
|
|
+ 2*BLOCK_SIZE, null, policyMap);
|
|
|
+ ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
|
|
|
+ NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES), null);
|
|
|
+ new MigrationTest(clusterScheme, nsScheme).runBasicTest(true);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Test directories with Hot, Warm and Cold polices.
|
|
|
*/
|
|
|
@Test
|
|
|
public void testHotWarmColdDirs() throws Exception {
|
|
|
+ LOG.info("testHotWarmColdDirs");
|
|
|
PathPolicyMap pathPolicyMap = new PathPolicyMap(3);
|
|
|
NamespaceScheme nsScheme = pathPolicyMap.newNamespaceScheme();
|
|
|
ClusterScheme clusterScheme = new ClusterScheme();
|
|
|
MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
|
|
|
|
|
|
- test.runBasicTest(false);
|
|
|
+ try {
|
|
|
+ test.runBasicTest(false);
|
|
|
+ pathPolicyMap.moveAround(test.dfs);
|
|
|
+ test.migrate();
|
|
|
|
|
|
- pathPolicyMap.moveAround(test.dfs);
|
|
|
- test.migrate();
|
|
|
- test.verify(true);
|
|
|
- test.shutdownCluster();
|
|
|
+ test.verify(true);
|
|
|
+ } finally {
|
|
|
+ test.shutdownCluster();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -495,76 +516,81 @@ public class TestStorageMover {
|
|
|
*/
|
|
|
@Test
|
|
|
public void testNoSpaceDisk() throws Exception {
|
|
|
+ LOG.info("testNoSpaceDisk");
|
|
|
final PathPolicyMap pathPolicyMap = new PathPolicyMap(0);
|
|
|
final NamespaceScheme nsScheme = pathPolicyMap.newNamespaceScheme();
|
|
|
|
|
|
- final long diskCapacity = (10 + HdfsConstants.MIN_BLOCKS_FOR_WRITE)*BLOCK_SIZE;
|
|
|
- final long archiveCapacity = 100*BLOCK_SIZE;
|
|
|
+ final long diskCapacity = (6 + HdfsConstants.MIN_BLOCKS_FOR_WRITE)
|
|
|
+ * BLOCK_SIZE;
|
|
|
+ final long archiveCapacity = 100 * BLOCK_SIZE;
|
|
|
final long[][] capacities = genCapacities(NUM_DATANODES, 1, 1,
|
|
|
diskCapacity, archiveCapacity);
|
|
|
- final ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
|
|
|
+ Configuration conf = new Configuration(DEFAULT_CONF);
|
|
|
+ final ClusterScheme clusterScheme = new ClusterScheme(conf,
|
|
|
NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES, 1, 1), capacities);
|
|
|
final MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
|
|
|
|
|
|
- test.runBasicTest(false);
|
|
|
-
|
|
|
- // create hot files with replication 3 until not more spaces.
|
|
|
- final short replication = 3;
|
|
|
- {
|
|
|
- int hotFileCount = 0;
|
|
|
- try {
|
|
|
- for(; ; hotFileCount++) {
|
|
|
- final Path p = new Path(pathPolicyMap.hot, "file" + hotFileCount);
|
|
|
- DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
|
|
|
+ try {
|
|
|
+ test.runBasicTest(false);
|
|
|
+
|
|
|
+ // create hot files with replication 3 until not more spaces.
|
|
|
+ final short replication = 3;
|
|
|
+ {
|
|
|
+ int hotFileCount = 0;
|
|
|
+ try {
|
|
|
+ for (; ; hotFileCount++) {
|
|
|
+ final Path p = new Path(pathPolicyMap.hot, "file" + hotFileCount);
|
|
|
+ DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.info("Expected: hotFileCount=" + hotFileCount, e);
|
|
|
}
|
|
|
- } catch(IOException e) {
|
|
|
- LOG.info("Expected: hotFileCount=" + hotFileCount, e);
|
|
|
+ Assert.assertTrue(hotFileCount >= 1);
|
|
|
}
|
|
|
- Assert.assertTrue(hotFileCount >= 2);
|
|
|
- }
|
|
|
|
|
|
- // create hot files with replication 1 to use up all remaining spaces.
|
|
|
- {
|
|
|
- int hotFileCount_r1 = 0;
|
|
|
- try {
|
|
|
- for(; ; hotFileCount_r1++) {
|
|
|
- final Path p = new Path(pathPolicyMap.hot, "file_r1_" + hotFileCount_r1);
|
|
|
- DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short)1, 0L);
|
|
|
+ // create hot files with replication 1 to use up all remaining spaces.
|
|
|
+ {
|
|
|
+ int hotFileCount_r1 = 0;
|
|
|
+ try {
|
|
|
+ for (; ; hotFileCount_r1++) {
|
|
|
+ final Path p = new Path(pathPolicyMap.hot, "file_r1_" + hotFileCount_r1);
|
|
|
+ DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short) 1, 0L);
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.info("Expected: hotFileCount_r1=" + hotFileCount_r1, e);
|
|
|
}
|
|
|
- } catch(IOException e) {
|
|
|
- LOG.info("Expected: hotFileCount_r1=" + hotFileCount_r1, e);
|
|
|
}
|
|
|
- }
|
|
|
|
|
|
- { // test increasing replication. Since DISK is full,
|
|
|
- // new replicas should be stored in ARCHIVE as a fallback storage.
|
|
|
- final Path file0 = new Path(pathPolicyMap.hot, "file0");
|
|
|
- final Replication r = test.getReplication(file0);
|
|
|
- final short newReplication = (short)5;
|
|
|
- test.dfs.setReplication(file0, newReplication);
|
|
|
- Thread.sleep(10000);
|
|
|
- test.verifyReplication(file0, r.disk, newReplication - r.disk);
|
|
|
- }
|
|
|
+ { // test increasing replication. Since DISK is full,
|
|
|
+ // new replicas should be stored in ARCHIVE as a fallback storage.
|
|
|
+ final Path file0 = new Path(pathPolicyMap.hot, "file0");
|
|
|
+ final Replication r = test.getReplication(file0);
|
|
|
+ final short newReplication = (short) 5;
|
|
|
+ test.dfs.setReplication(file0, newReplication);
|
|
|
+ Thread.sleep(10000);
|
|
|
+ test.verifyReplication(file0, r.disk, newReplication - r.disk);
|
|
|
+ }
|
|
|
|
|
|
- { // test creating a cold file and then increase replication
|
|
|
- final Path p = new Path(pathPolicyMap.cold, "foo");
|
|
|
- DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
|
|
|
- test.verifyReplication(p, 0, replication);
|
|
|
+ { // test creating a cold file and then increase replication
|
|
|
+ final Path p = new Path(pathPolicyMap.cold, "foo");
|
|
|
+ DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
|
|
|
+ test.verifyReplication(p, 0, replication);
|
|
|
|
|
|
- final short newReplication = 5;
|
|
|
- test.dfs.setReplication(p, newReplication);
|
|
|
- Thread.sleep(10000);
|
|
|
- test.verifyReplication(p, 0, newReplication);
|
|
|
- }
|
|
|
+ final short newReplication = 5;
|
|
|
+ test.dfs.setReplication(p, newReplication);
|
|
|
+ Thread.sleep(10000);
|
|
|
+ test.verifyReplication(p, 0, newReplication);
|
|
|
+ }
|
|
|
|
|
|
- { //test move a hot file to warm
|
|
|
- final Path file1 = new Path(pathPolicyMap.hot, "file1");
|
|
|
- test.dfs.rename(file1, pathPolicyMap.warm);
|
|
|
- test.migrate();
|
|
|
- test.verifyFile(new Path(pathPolicyMap.warm, "file1"), WARM.getId());;
|
|
|
+ { //test move a hot file to warm
|
|
|
+ final Path file1 = new Path(pathPolicyMap.hot, "file1");
|
|
|
+ test.dfs.rename(file1, pathPolicyMap.warm);
|
|
|
+ test.migrate();
|
|
|
+ test.verifyFile(new Path(pathPolicyMap.warm, "file1"), WARM.getId());
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ test.shutdownCluster();
|
|
|
}
|
|
|
-
|
|
|
- test.shutdownCluster();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -572,73 +598,77 @@ public class TestStorageMover {
|
|
|
*/
|
|
|
@Test
|
|
|
public void testNoSpaceArchive() throws Exception {
|
|
|
+ LOG.info("testNoSpaceArchive");
|
|
|
final PathPolicyMap pathPolicyMap = new PathPolicyMap(0);
|
|
|
final NamespaceScheme nsScheme = pathPolicyMap.newNamespaceScheme();
|
|
|
|
|
|
- final long diskCapacity = 100*BLOCK_SIZE;
|
|
|
- final long archiveCapacity = (10 + HdfsConstants.MIN_BLOCKS_FOR_WRITE)*BLOCK_SIZE;
|
|
|
+ final long diskCapacity = 100 * BLOCK_SIZE;
|
|
|
+ final long archiveCapacity = (6 + HdfsConstants.MIN_BLOCKS_FOR_WRITE)
|
|
|
+ * BLOCK_SIZE;
|
|
|
final long[][] capacities = genCapacities(NUM_DATANODES, 1, 1,
|
|
|
diskCapacity, archiveCapacity);
|
|
|
final ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
|
|
|
NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES, 1, 1), capacities);
|
|
|
final MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
|
|
|
|
|
|
- test.runBasicTest(false);
|
|
|
-
|
|
|
- // create cold files with replication 3 until not more spaces.
|
|
|
- final short replication = 3;
|
|
|
- {
|
|
|
- int coldFileCount = 0;
|
|
|
- try {
|
|
|
- for(; ; coldFileCount++) {
|
|
|
- final Path p = new Path(pathPolicyMap.cold, "file" + coldFileCount);
|
|
|
- DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
|
|
|
+ try {
|
|
|
+ test.runBasicTest(false);
|
|
|
+
|
|
|
+ // create cold files with replication 3 until not more spaces.
|
|
|
+ final short replication = 3;
|
|
|
+ {
|
|
|
+ int coldFileCount = 0;
|
|
|
+ try {
|
|
|
+ for (; ; coldFileCount++) {
|
|
|
+ final Path p = new Path(pathPolicyMap.cold, "file" + coldFileCount);
|
|
|
+ DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.info("Expected: coldFileCount=" + coldFileCount, e);
|
|
|
}
|
|
|
- } catch(IOException e) {
|
|
|
- LOG.info("Expected: coldFileCount=" + coldFileCount, e);
|
|
|
+ Assert.assertTrue(coldFileCount >= 1);
|
|
|
}
|
|
|
- Assert.assertTrue(coldFileCount >= 2);
|
|
|
- }
|
|
|
|
|
|
- // create cold files with replication 1 to use up all remaining spaces.
|
|
|
- {
|
|
|
- int coldFileCount_r1 = 0;
|
|
|
- try {
|
|
|
- for(; ; coldFileCount_r1++) {
|
|
|
- final Path p = new Path(pathPolicyMap.cold, "file_r1_" + coldFileCount_r1);
|
|
|
- DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short)1, 0L);
|
|
|
+ // create cold files with replication 1 to use up all remaining spaces.
|
|
|
+ {
|
|
|
+ int coldFileCount_r1 = 0;
|
|
|
+ try {
|
|
|
+ for (; ; coldFileCount_r1++) {
|
|
|
+ final Path p = new Path(pathPolicyMap.cold, "file_r1_" + coldFileCount_r1);
|
|
|
+ DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short) 1, 0L);
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.info("Expected: coldFileCount_r1=" + coldFileCount_r1, e);
|
|
|
}
|
|
|
- } catch(IOException e) {
|
|
|
- LOG.info("Expected: coldFileCount_r1=" + coldFileCount_r1, e);
|
|
|
}
|
|
|
- }
|
|
|
|
|
|
- { // test increasing replication but new replicas cannot be created
|
|
|
- // since no more ARCHIVE space.
|
|
|
- final Path file0 = new Path(pathPolicyMap.cold, "file0");
|
|
|
- final Replication r = test.getReplication(file0);
|
|
|
- LOG.info("XXX " + file0 + ": replication=" + r);
|
|
|
- Assert.assertEquals(0, r.disk);
|
|
|
+ { // test increasing replication but new replicas cannot be created
|
|
|
+ // since no more ARCHIVE space.
|
|
|
+ final Path file0 = new Path(pathPolicyMap.cold, "file0");
|
|
|
+ final Replication r = test.getReplication(file0);
|
|
|
+ LOG.info("XXX " + file0 + ": replication=" + r);
|
|
|
+ Assert.assertEquals(0, r.disk);
|
|
|
|
|
|
- final short newReplication = (short)5;
|
|
|
- test.dfs.setReplication(file0, newReplication);
|
|
|
- Thread.sleep(10000);
|
|
|
+ final short newReplication = (short) 5;
|
|
|
+ test.dfs.setReplication(file0, newReplication);
|
|
|
+ Thread.sleep(10000);
|
|
|
|
|
|
- test.verifyReplication(file0, 0, r.archive);
|
|
|
- }
|
|
|
+ test.verifyReplication(file0, 0, r.archive);
|
|
|
+ }
|
|
|
|
|
|
- { // test creating a hot file
|
|
|
- final Path p = new Path(pathPolicyMap.hot, "foo");
|
|
|
- DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short)3, 0L);
|
|
|
- }
|
|
|
+ { // test creating a hot file
|
|
|
+ final Path p = new Path(pathPolicyMap.hot, "foo");
|
|
|
+ DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short) 3, 0L);
|
|
|
+ }
|
|
|
|
|
|
- { //test move a cold file to warm
|
|
|
- final Path file1 = new Path(pathPolicyMap.hot, "file1");
|
|
|
- test.dfs.rename(file1, pathPolicyMap.warm);
|
|
|
- test.migrate();
|
|
|
- test.verify(true);
|
|
|
+ { //test move a cold file to warm
|
|
|
+ final Path file1 = new Path(pathPolicyMap.cold, "file1");
|
|
|
+ test.dfs.rename(file1, pathPolicyMap.warm);
|
|
|
+ test.migrate();
|
|
|
+ test.verify(true);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ test.shutdownCluster();
|
|
|
}
|
|
|
-
|
|
|
- test.shutdownCluster();
|
|
|
}
|
|
|
}
|