|
@@ -58,6 +58,7 @@ import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
@@ -239,12 +240,17 @@ public abstract class LazyPersistTestCase {
|
|
|
long ramDiskStorageLimit,
|
|
|
long evictionLowWatermarkReplicas,
|
|
|
boolean useSCR,
|
|
|
- boolean useLegacyBlockReaderLocal) throws IOException {
|
|
|
+ boolean useLegacyBlockReaderLocal,
|
|
|
+ boolean disableScrubber) throws IOException {
|
|
|
|
|
|
Configuration conf = new Configuration();
|
|
|
conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
- conf.setInt(DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC,
|
|
|
- LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC);
|
|
|
+ if (disableScrubber) {
|
|
|
+ conf.setInt(DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC, 0);
|
|
|
+ } else {
|
|
|
+ conf.setInt(DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC,
|
|
|
+ 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);
|
|
@@ -357,11 +363,16 @@ public abstract class LazyPersistTestCase {
|
|
|
return this;
|
|
|
}
|
|
|
|
|
|
+ public ClusterWithRamDiskBuilder disableScrubber() {
|
|
|
+ this.disableScrubber = true;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
public void build() throws IOException {
|
|
|
LazyPersistTestCase.this.startUpCluster(
|
|
|
numDatanodes, hasTransientStorage, storageTypes, ramDiskReplicaCapacity,
|
|
|
ramDiskStorageLimit, evictionLowWatermarkReplicas,
|
|
|
- useScr, useLegacyBlockReaderLocal);
|
|
|
+ useScr, useLegacyBlockReaderLocal,disableScrubber);
|
|
|
}
|
|
|
|
|
|
private int numDatanodes = REPL_FACTOR;
|
|
@@ -372,6 +383,7 @@ public abstract class LazyPersistTestCase {
|
|
|
private boolean useScr = false;
|
|
|
private boolean useLegacyBlockReaderLocal = false;
|
|
|
private long evictionLowWatermarkReplicas = EVICTION_LOW_WATERMARK;
|
|
|
+ private boolean disableScrubber=false;
|
|
|
}
|
|
|
|
|
|
protected final void triggerBlockReport()
|