|
@@ -23,16 +23,7 @@ import static org.apache.hadoop.fs.CreateFlag.CREATE;
|
|
import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
|
|
import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
|
|
import static org.apache.hadoop.fs.StorageType.DEFAULT;
|
|
import static org.apache.hadoop.fs.StorageType.DEFAULT;
|
|
import static org.apache.hadoop.fs.StorageType.RAM_DISK;
|
|
import static org.apache.hadoop.fs.StorageType.RAM_DISK;
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
|
import static org.hamcrest.core.Is.is;
|
|
import static org.hamcrest.core.Is.is;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertThat;
|
|
import static org.junit.Assert.assertThat;
|
|
@@ -40,6 +31,7 @@ import static org.junit.Assert.fail;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.nio.ByteBuffer;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.EnumSet;
|
|
import java.util.EnumSet;
|
|
import java.util.HashSet;
|
|
import java.util.HashSet;
|
|
@@ -68,6 +60,7 @@ import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
import org.apache.hadoop.hdfs.tools.JMXGet;
|
|
import org.apache.hadoop.hdfs.tools.JMXGet;
|
|
|
|
+import org.apache.hadoop.io.nativeio.NativeIO;
|
|
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
|
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
@@ -80,8 +73,8 @@ public abstract class LazyPersistTestCase {
|
|
static final byte LAZY_PERSIST_POLICY_ID = (byte) 15;
|
|
static final byte LAZY_PERSIST_POLICY_ID = (byte) 15;
|
|
|
|
|
|
static {
|
|
static {
|
|
- DFSTestUtil.setNameNodeLogLevel(Level.ALL);
|
|
|
|
- GenericTestUtils.setLogLevel(FsDatasetImpl.LOG, Level.ALL);
|
|
|
|
|
|
+ DFSTestUtil.setNameNodeLogLevel(Level.DEBUG);
|
|
|
|
+ GenericTestUtils.setLogLevel(FsDatasetImpl.LOG, Level.DEBUG);
|
|
}
|
|
}
|
|
|
|
|
|
protected static final int BLOCK_SIZE = 5 * 1024 * 1024;
|
|
protected static final int BLOCK_SIZE = 5 * 1024 * 1024;
|
|
@@ -95,6 +88,8 @@ public abstract class LazyPersistTestCase {
|
|
protected static final int LAZY_WRITER_INTERVAL_SEC = 1;
|
|
protected static final int LAZY_WRITER_INTERVAL_SEC = 1;
|
|
protected static final Log LOG = LogFactory.getLog(LazyPersistTestCase.class);
|
|
protected static final Log LOG = LogFactory.getLog(LazyPersistTestCase.class);
|
|
protected static final short REPL_FACTOR = 1;
|
|
protected static final short REPL_FACTOR = 1;
|
|
|
|
+ protected final long osPageSize =
|
|
|
|
+ NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize();
|
|
|
|
|
|
protected MiniDFSCluster cluster;
|
|
protected MiniDFSCluster cluster;
|
|
protected DistributedFileSystem fs;
|
|
protected DistributedFileSystem fs;
|
|
@@ -194,7 +189,7 @@ public abstract class LazyPersistTestCase {
|
|
protected final void makeRandomTestFile(Path path, long length,
|
|
protected final void makeRandomTestFile(Path path, long length,
|
|
boolean isLazyPersist, long seed) throws IOException {
|
|
boolean isLazyPersist, long seed) throws IOException {
|
|
DFSTestUtil.createFile(fs, path, isLazyPersist, BUFFER_LENGTH, length,
|
|
DFSTestUtil.createFile(fs, path, isLazyPersist, BUFFER_LENGTH, length,
|
|
- BLOCK_SIZE, REPL_FACTOR, seed, true);
|
|
|
|
|
|
+ BLOCK_SIZE, REPL_FACTOR, seed, true);
|
|
}
|
|
}
|
|
|
|
|
|
protected final void makeTestFile(Path path, long length,
|
|
protected final void makeTestFile(Path path, long length,
|
|
@@ -242,10 +237,12 @@ public abstract class LazyPersistTestCase {
|
|
int ramDiskReplicaCapacity,
|
|
int ramDiskReplicaCapacity,
|
|
long ramDiskStorageLimit,
|
|
long ramDiskStorageLimit,
|
|
long evictionLowWatermarkReplicas,
|
|
long evictionLowWatermarkReplicas,
|
|
|
|
+ long maxLockedMemory,
|
|
boolean useSCR,
|
|
boolean useSCR,
|
|
boolean useLegacyBlockReaderLocal,
|
|
boolean useLegacyBlockReaderLocal,
|
|
boolean disableScrubber) throws IOException {
|
|
boolean disableScrubber) throws IOException {
|
|
|
|
|
|
|
|
+ initCacheManipulator();
|
|
Configuration conf = new Configuration();
|
|
Configuration conf = new Configuration();
|
|
conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
if (disableScrubber) {
|
|
if (disableScrubber) {
|
|
@@ -262,6 +259,7 @@ public abstract class LazyPersistTestCase {
|
|
conf.setLong(DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES,
|
|
conf.setLong(DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES,
|
|
evictionLowWatermarkReplicas * BLOCK_SIZE);
|
|
evictionLowWatermarkReplicas * BLOCK_SIZE);
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
|
|
|
|
+ conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, maxLockedMemory);
|
|
|
|
|
|
if (useSCR) {
|
|
if (useSCR) {
|
|
conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
|
|
conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
|
|
@@ -311,6 +309,31 @@ public abstract class LazyPersistTestCase {
|
|
LOG.info("Cluster startup complete");
|
|
LOG.info("Cluster startup complete");
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Use a dummy cache manipulator for testing.
|
|
|
|
+ */
|
|
|
|
+ public static void initCacheManipulator() {
|
|
|
|
+ NativeIO.POSIX.setCacheManipulator(new NativeIO.POSIX.CacheManipulator() {
|
|
|
|
+ @Override
|
|
|
|
+ public void mlock(String identifier,
|
|
|
|
+ ByteBuffer mmap, long length) throws IOException {
|
|
|
|
+ LOG.info("LazyPersistTestCase: faking mlock of " + identifier + " bytes.");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public long getMemlockLimit() {
|
|
|
|
+ LOG.info("LazyPersistTestCase: fake return " + Long.MAX_VALUE);
|
|
|
|
+ return Long.MAX_VALUE;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public boolean verifyCanMlock() {
|
|
|
|
+ LOG.info("LazyPersistTestCase: fake return " + true);
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+ }
|
|
|
|
+
|
|
ClusterWithRamDiskBuilder getClusterBuilder() {
|
|
ClusterWithRamDiskBuilder getClusterBuilder() {
|
|
return new ClusterWithRamDiskBuilder();
|
|
return new ClusterWithRamDiskBuilder();
|
|
}
|
|
}
|
|
@@ -344,6 +367,11 @@ public abstract class LazyPersistTestCase {
|
|
return this;
|
|
return this;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public ClusterWithRamDiskBuilder setMaxLockedMemory(long maxLockedMemory) {
|
|
|
|
+ this.maxLockedMemory = maxLockedMemory;
|
|
|
|
+ return this;
|
|
|
|
+ }
|
|
|
|
+
|
|
public ClusterWithRamDiskBuilder setUseScr(boolean useScr) {
|
|
public ClusterWithRamDiskBuilder setUseScr(boolean useScr) {
|
|
this.useScr = useScr;
|
|
this.useScr = useScr;
|
|
return this;
|
|
return this;
|
|
@@ -376,13 +404,14 @@ public abstract class LazyPersistTestCase {
|
|
LazyPersistTestCase.this.startUpCluster(
|
|
LazyPersistTestCase.this.startUpCluster(
|
|
numDatanodes, hasTransientStorage, storageTypes, ramDiskReplicaCapacity,
|
|
numDatanodes, hasTransientStorage, storageTypes, ramDiskReplicaCapacity,
|
|
ramDiskStorageLimit, evictionLowWatermarkReplicas,
|
|
ramDiskStorageLimit, evictionLowWatermarkReplicas,
|
|
- useScr, useLegacyBlockReaderLocal,disableScrubber);
|
|
|
|
|
|
+ maxLockedMemory, useScr, useLegacyBlockReaderLocal, disableScrubber);
|
|
}
|
|
}
|
|
|
|
|
|
private int numDatanodes = REPL_FACTOR;
|
|
private int numDatanodes = REPL_FACTOR;
|
|
private StorageType[] storageTypes = null;
|
|
private StorageType[] storageTypes = null;
|
|
private int ramDiskReplicaCapacity = -1;
|
|
private int ramDiskReplicaCapacity = -1;
|
|
private long ramDiskStorageLimit = -1;
|
|
private long ramDiskStorageLimit = -1;
|
|
|
|
+ private long maxLockedMemory = Long.MAX_VALUE;
|
|
private boolean hasTransientStorage = true;
|
|
private boolean hasTransientStorage = true;
|
|
private boolean useScr = false;
|
|
private boolean useScr = false;
|
|
private boolean useLegacyBlockReaderLocal = false;
|
|
private boolean useLegacyBlockReaderLocal = false;
|