|
@@ -23,7 +23,6 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
-import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
@@ -71,13 +70,14 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
private static final int THREADPOOL_SIZE = 10;
|
|
|
|
|
|
- private static short REPL_FACTOR = 1;
|
|
|
+ private static final short REPL_FACTOR = 1;
|
|
|
private static final int BLOCK_SIZE = 10485760; // 10 MB
|
|
|
private static final int LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC = 3;
|
|
|
private static final long HEARTBEAT_INTERVAL_SEC = 1;
|
|
|
private static final int HEARTBEAT_RECHECK_INTERVAL_MSEC = 500;
|
|
|
private static final int LAZY_WRITER_INTERVAL_SEC = 1;
|
|
|
private static final int BUFFER_LENGTH = 4096;
|
|
|
+ private static final int EVICTION_LOW_WATERMARK = 1;
|
|
|
|
|
|
private MiniDFSCluster cluster;
|
|
|
private DistributedFileSystem fs;
|
|
@@ -101,7 +101,7 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
@Test (timeout=300000)
|
|
|
public void testFlagNotSetByDefault() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, null, -1);
|
|
|
+ startUpCluster(false, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -113,7 +113,7 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
@Test (timeout=300000)
|
|
|
public void testFlagPropagation() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, null, -1);
|
|
|
+ startUpCluster(false, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -125,7 +125,7 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
@Test (timeout=300000)
|
|
|
public void testFlagPersistenceInEditLog() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, null, -1);
|
|
|
+ startUpCluster(false, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -139,10 +139,9 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
@Test (timeout=300000)
|
|
|
public void testFlagPersistenceInFsImage() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, null, -1);
|
|
|
+ startUpCluster(false, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
- FSDataOutputStream fos = null;
|
|
|
|
|
|
makeTestFile(path, 0, true);
|
|
|
// checkpoint
|
|
@@ -158,7 +157,7 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
@Test (timeout=300000)
|
|
|
public void testPlacementOnRamDisk() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { DEFAULT, RAM_DISK}, -1);
|
|
|
+ startUpCluster(true, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -168,8 +167,7 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
@Test (timeout=300000)
|
|
|
public void testPlacementOnSizeLimitedRamDisk() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { DEFAULT, RAM_DISK },
|
|
|
- 3 * BLOCK_SIZE -1); // 2 replicas + delta
|
|
|
+ startUpCluster(true, 3);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
|
|
@@ -188,7 +186,7 @@ public class TestLazyPersistFiles {
|
|
|
*/
|
|
|
@Test (timeout=300000)
|
|
|
public void testFallbackToDisk() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, null, -1);
|
|
|
+ startUpCluster(false, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -202,7 +200,7 @@ public class TestLazyPersistFiles {
|
|
|
*/
|
|
|
@Test (timeout=300000)
|
|
|
public void testFallbackToDiskFull() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, null, BLOCK_SIZE - 1);
|
|
|
+ startUpCluster(false, 0);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -213,15 +211,13 @@ public class TestLazyPersistFiles {
|
|
|
/**
|
|
|
* File partially fit in RamDisk after eviction.
|
|
|
* RamDisk can fit 2 blocks. Write a file with 5 blocks.
|
|
|
- * Expect 2 blocks are on RamDisk whereas other 3 on disk.
|
|
|
+ * Expect 2 or less blocks are on RamDisk and 3 or more on disk.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
@Test (timeout=300000)
|
|
|
public void testFallbackToDiskPartial()
|
|
|
throws IOException, InterruptedException {
|
|
|
- startUpCluster(REPL_FACTOR,
|
|
|
- new StorageType[] { RAM_DISK, DEFAULT },
|
|
|
- BLOCK_SIZE * 3 - 1);
|
|
|
+ startUpCluster(true, 2);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -241,12 +237,15 @@ public class TestLazyPersistFiles {
|
|
|
for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
|
|
|
if (locatedBlock.getStorageTypes()[0] == RAM_DISK) {
|
|
|
numBlocksOnRamDisk++;
|
|
|
- }else if (locatedBlock.getStorageTypes()[0] == DEFAULT) {
|
|
|
+ } else if (locatedBlock.getStorageTypes()[0] == DEFAULT) {
|
|
|
numBlocksOnDisk++;
|
|
|
}
|
|
|
}
|
|
|
- assertThat(numBlocksOnRamDisk, is(2));
|
|
|
- assertThat(numBlocksOnDisk, is(3));
|
|
|
+
|
|
|
+ // Since eviction is asynchronous, depending on the timing of eviction
|
|
|
+ // wrt writes, we may get 2 or less blocks on RAM disk.
|
|
|
+ assert(numBlocksOnRamDisk <= 2);
|
|
|
+ assert(numBlocksOnDisk >= 3);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -257,7 +256,7 @@ public class TestLazyPersistFiles {
|
|
|
*/
|
|
|
@Test (timeout=300000)
|
|
|
public void testRamDiskNotChosenByDefault() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] {RAM_DISK, RAM_DISK}, -1);
|
|
|
+ startUpCluster(true, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -275,7 +274,7 @@ public class TestLazyPersistFiles {
|
|
|
*/
|
|
|
@Test (timeout=300000)
|
|
|
public void testAppendIsDenied() throws IOException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] {RAM_DISK, DEFAULT }, -1);
|
|
|
+ startUpCluster(true, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -297,17 +296,12 @@ public class TestLazyPersistFiles {
|
|
|
@Test (timeout=300000)
|
|
|
public void testLazyPersistFilesAreDiscarded()
|
|
|
throws IOException, InterruptedException {
|
|
|
- startUpCluster(REPL_FACTOR,
|
|
|
- new StorageType[] { RAM_DISK, DEFAULT },
|
|
|
- (2 * BLOCK_SIZE - 1)); // 1 replica + delta.
|
|
|
+ startUpCluster(true, 2);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
- Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
|
|
|
|
|
|
makeTestFile(path1, BLOCK_SIZE, true);
|
|
|
- makeTestFile(path2, BLOCK_SIZE, false);
|
|
|
ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
- ensureFileReplicasOnStorageType(path2, DEFAULT);
|
|
|
|
|
|
// Stop the DataNode and sleep for the time it takes the NN to
|
|
|
// detect the DN as being dead.
|
|
@@ -315,30 +309,28 @@ public class TestLazyPersistFiles {
|
|
|
Thread.sleep(30000L);
|
|
|
assertThat(cluster.getNamesystem().getNumDeadDataNodes(), is(1));
|
|
|
|
|
|
- // Next, wait for the replication monitor to mark the file as
|
|
|
- // corrupt, plus some delta.
|
|
|
+ // Next, wait for the replication monitor to mark the file as corrupt
|
|
|
Thread.sleep(2 * DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT * 1000);
|
|
|
|
|
|
- // Wait for the LazyPersistFileScrubber to run, plus some delta.
|
|
|
+ // Wait for the LazyPersistFileScrubber to run
|
|
|
Thread.sleep(2 * LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC * 1000);
|
|
|
|
|
|
// Ensure that path1 does not exist anymore, whereas path2 does.
|
|
|
assert(!fs.exists(path1));
|
|
|
- assert(fs.exists(path2));
|
|
|
|
|
|
- // We should have only one block that needs replication i.e. the one
|
|
|
+ // We should have zero blocks that needs replication i.e. the one
|
|
|
// belonging to path2.
|
|
|
assertThat(cluster.getNameNode()
|
|
|
.getNamesystem()
|
|
|
.getBlockManager()
|
|
|
.getUnderReplicatedBlocksCount(),
|
|
|
- is(1L));
|
|
|
+ is(0L));
|
|
|
}
|
|
|
|
|
|
@Test (timeout=300000)
|
|
|
public void testLazyPersistBlocksAreSaved()
|
|
|
throws IOException, InterruptedException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { RAM_DISK, DEFAULT }, -1);
|
|
|
+ startUpCluster(true, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -386,16 +378,12 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
/**
|
|
|
* RamDisk eviction after lazy persist to disk.
|
|
|
- * Evicted blocks are still readable with on-disk replicas.
|
|
|
* @throws IOException
|
|
|
* @throws InterruptedException
|
|
|
*/
|
|
|
- @Test (timeout=300000)
|
|
|
- public void testRamDiskEviction()
|
|
|
- throws IOException, InterruptedException {
|
|
|
- startUpCluster(REPL_FACTOR,
|
|
|
- new StorageType[] { RAM_DISK, DEFAULT },
|
|
|
- (2 * BLOCK_SIZE - 1)); // 1 replica + delta.
|
|
|
+ @Test (timeout=300000)
|
|
|
+ public void testRamDiskEviction() throws IOException, InterruptedException {
|
|
|
+ startUpCluster(true, 1 + EVICTION_LOW_WATERMARK);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
|
|
@@ -405,16 +393,16 @@ public class TestLazyPersistFiles {
|
|
|
ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
|
|
|
// Sleep for a short time to allow the lazy writer thread to do its job.
|
|
|
- // However the block replica should not be evicted from RAM_DISK yet.
|
|
|
Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
|
|
|
ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
|
|
|
// Create another file with a replica on RAM_DISK.
|
|
|
makeTestFile(path2, BLOCK_SIZE, true);
|
|
|
+ Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
|
|
|
triggerBlockReport();
|
|
|
|
|
|
- // Make sure that the second file's block replica is on RAM_DISK, whereas
|
|
|
- // the original file's block replica is now on disk.
|
|
|
+ // Ensure the first file was evicted to disk, the second is still on
|
|
|
+ // RAM_DISK.
|
|
|
ensureFileReplicasOnStorageType(path2, RAM_DISK);
|
|
|
ensureFileReplicasOnStorageType(path1, DEFAULT);
|
|
|
}
|
|
@@ -428,9 +416,7 @@ public class TestLazyPersistFiles {
|
|
|
@Test (timeout=300000)
|
|
|
public void testRamDiskEvictionBeforePersist()
|
|
|
throws IOException, InterruptedException {
|
|
|
- // 1 replica + delta, lazy persist interval every 50 minutes
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { RAM_DISK, DEFAULT },
|
|
|
- (2 * BLOCK_SIZE - 1));
|
|
|
+ startUpCluster(true, 1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
|
|
@@ -463,8 +449,7 @@ public class TestLazyPersistFiles {
|
|
|
@Test (timeout=300000)
|
|
|
public void testRamDiskEvictionLRU()
|
|
|
throws IOException, InterruptedException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { RAM_DISK, DEFAULT },
|
|
|
- (4 * BLOCK_SIZE -1)); // 3 replica + delta.
|
|
|
+ startUpCluster(true, 3);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
final int NUM_PATHS = 6;
|
|
|
Path paths[] = new Path[NUM_PATHS];
|
|
@@ -501,8 +486,7 @@ public class TestLazyPersistFiles {
|
|
|
@Test (timeout=300000)
|
|
|
public void testDeleteBeforePersist()
|
|
|
throws IOException, InterruptedException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { RAM_DISK, DEFAULT },
|
|
|
- -1);
|
|
|
+ startUpCluster(true, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
stopLazyWriter(cluster.getDataNodes().get(0));
|
|
|
|
|
@@ -527,7 +511,7 @@ public class TestLazyPersistFiles {
|
|
|
@Test (timeout=300000)
|
|
|
public void testDeleteAfterPersist()
|
|
|
throws IOException, InterruptedException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { RAM_DISK, DEFAULT }, -1);
|
|
|
+ startUpCluster(true, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -554,8 +538,7 @@ public class TestLazyPersistFiles {
|
|
|
@Test (timeout=300000)
|
|
|
public void testDfsUsageCreateDelete()
|
|
|
throws IOException, InterruptedException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { RAM_DISK, DEFAULT },
|
|
|
- 5 * BLOCK_SIZE - 1); // 4 replica + delta
|
|
|
+ startUpCluster(true, 4);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -586,8 +569,7 @@ public class TestLazyPersistFiles {
|
|
|
@Test (timeout=300000)
|
|
|
public void testConcurrentRead()
|
|
|
throws Exception {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { DEFAULT, RAM_DISK },
|
|
|
- 3 * BLOCK_SIZE -1); // 2 replicas + delta
|
|
|
+ startUpCluster(true, 2);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
final Path path1 = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
@@ -638,8 +620,7 @@ public class TestLazyPersistFiles {
|
|
|
@Test (timeout=300000)
|
|
|
public void testConcurrentWrites()
|
|
|
throws IOException, InterruptedException {
|
|
|
- startUpCluster(REPL_FACTOR, new StorageType[] { RAM_DISK, DEFAULT },
|
|
|
- (10 * BLOCK_SIZE -1)); // 9 replica + delta.
|
|
|
+ startUpCluster(true, 9);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
final int SEED = 0xFADED;
|
|
|
final int NUM_WRITERS = 4;
|
|
@@ -659,8 +640,7 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
ExecutorService executor = Executors.newFixedThreadPool(THREADPOOL_SIZE);
|
|
|
for (int i = 0; i < NUM_WRITERS; i++) {
|
|
|
- Runnable writer = new WriterRunnable(cluster, i, paths[i], SEED, latch,
|
|
|
- testFailed);
|
|
|
+ Runnable writer = new WriterRunnable(i, paths[i], SEED, latch, testFailed);
|
|
|
executor.execute(writer);
|
|
|
}
|
|
|
|
|
@@ -677,9 +657,7 @@ public class TestLazyPersistFiles {
|
|
|
public void testDnRestartWithSavedReplicas()
|
|
|
throws IOException, InterruptedException {
|
|
|
|
|
|
- startUpCluster(REPL_FACTOR,
|
|
|
- new StorageType[] {RAM_DISK, DEFAULT },
|
|
|
- (2 * BLOCK_SIZE - 1)); // 1 replica + delta.
|
|
|
+ startUpCluster(true, -1);
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
|
|
@@ -703,9 +681,7 @@ public class TestLazyPersistFiles {
|
|
|
public void testDnRestartWithUnsavedReplicas()
|
|
|
throws IOException, InterruptedException {
|
|
|
|
|
|
- startUpCluster(REPL_FACTOR,
|
|
|
- new StorageType[] {RAM_DISK, DEFAULT },
|
|
|
- (2 * BLOCK_SIZE - 1)); // 1 replica + delta.
|
|
|
+ startUpCluster(true, 1);
|
|
|
stopLazyWriter(cluster.getDataNodes().get(0));
|
|
|
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
@@ -727,9 +703,8 @@ public class TestLazyPersistFiles {
|
|
|
* If ramDiskStorageLimit is >=0, then RAM_DISK capacity is artificially
|
|
|
* capped. If ramDiskStorageLimit < 0 then it is ignored.
|
|
|
*/
|
|
|
- private void startUpCluster(final int numDataNodes,
|
|
|
- final StorageType[] storageTypes,
|
|
|
- final long ramDiskStorageLimit,
|
|
|
+ private void startUpCluster(boolean hasTransientStorage,
|
|
|
+ final int ramDiskReplicaCapacity,
|
|
|
final boolean useSCR)
|
|
|
throws IOException {
|
|
|
|
|
@@ -739,42 +714,36 @@ public class TestLazyPersistFiles {
|
|
|
LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC);
|
|
|
conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL_SEC);
|
|
|
conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
|
|
|
- HEARTBEAT_RECHECK_INTERVAL_MSEC);
|
|
|
+ HEARTBEAT_RECHECK_INTERVAL_MSEC);
|
|
|
conf.setInt(DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC,
|
|
|
- LAZY_WRITER_INTERVAL_SEC);
|
|
|
+ LAZY_WRITER_INTERVAL_SEC);
|
|
|
+ conf.setInt(DFS_DATANODE_RAM_DISK_LOW_WATERMARK_REPLICAS,
|
|
|
+ EVICTION_LOW_WATERMARK);
|
|
|
|
|
|
- conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY,useSCR);
|
|
|
+ conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, useSCR);
|
|
|
|
|
|
- REPL_FACTOR = 1; //Reset in case a test has modified the value
|
|
|
+ long[] capacities = null;
|
|
|
+ if (hasTransientStorage && ramDiskReplicaCapacity >= 0) {
|
|
|
+ // Convert replica count to byte count, add some delta for .meta and VERSION files.
|
|
|
+ long ramDiskStorageLimit = ((long) ramDiskReplicaCapacity * BLOCK_SIZE) + (BLOCK_SIZE - 1);
|
|
|
+ capacities = new long[] { ramDiskStorageLimit, -1 };
|
|
|
+ }
|
|
|
|
|
|
cluster = new MiniDFSCluster
|
|
|
.Builder(conf)
|
|
|
- .numDataNodes(numDataNodes)
|
|
|
- .storageTypes(storageTypes != null ? storageTypes : new StorageType[] { DEFAULT, DEFAULT })
|
|
|
+ .numDataNodes(REPL_FACTOR)
|
|
|
+ .storageCapacities(capacities)
|
|
|
+ .storageTypes(hasTransientStorage ? new StorageType[]{ RAM_DISK, DEFAULT } : null)
|
|
|
.build();
|
|
|
fs = cluster.getFileSystem();
|
|
|
client = fs.getClient();
|
|
|
-
|
|
|
- // Artificially cap the storage capacity of the RAM_DISK volume.
|
|
|
- if (ramDiskStorageLimit >= 0) {
|
|
|
- List<? extends FsVolumeSpi> volumes =
|
|
|
- cluster.getDataNodes().get(0).getFSDataset().getVolumes();
|
|
|
-
|
|
|
- for (FsVolumeSpi volume : volumes) {
|
|
|
- if (volume.getStorageType() == RAM_DISK) {
|
|
|
- ((FsTransientVolumeImpl) volume).setCapacityForTesting(ramDiskStorageLimit);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
LOG.info("Cluster startup complete");
|
|
|
}
|
|
|
|
|
|
- private void startUpCluster(final int numDataNodes,
|
|
|
- final StorageType[] storageTypes,
|
|
|
- final long ramDiskStorageLimit)
|
|
|
+ private void startUpCluster(boolean hasTransientStorage,
|
|
|
+ final int ramDiskReplicaCapacity)
|
|
|
throws IOException {
|
|
|
- startUpCluster(numDataNodes, storageTypes, ramDiskStorageLimit, false);
|
|
|
+ startUpCluster(hasTransientStorage, ramDiskReplicaCapacity, false);
|
|
|
}
|
|
|
|
|
|
private void makeTestFile(Path path, long length, final boolean isLazyPersist)
|
|
@@ -908,17 +877,15 @@ public class TestLazyPersistFiles {
|
|
|
|
|
|
class WriterRunnable implements Runnable {
|
|
|
private final int id;
|
|
|
- private final MiniDFSCluster cluster;
|
|
|
private final Path paths[];
|
|
|
private final int seed;
|
|
|
private CountDownLatch latch;
|
|
|
private AtomicBoolean bFail;
|
|
|
|
|
|
- public WriterRunnable(MiniDFSCluster cluster, int threadIndex, Path[] paths,
|
|
|
+ public WriterRunnable(int threadIndex, Path[] paths,
|
|
|
int seed, CountDownLatch latch,
|
|
|
AtomicBoolean bFail) {
|
|
|
id = threadIndex;
|
|
|
- this.cluster = cluster;
|
|
|
this.paths = paths;
|
|
|
this.seed = seed;
|
|
|
this.latch = latch;
|