|
@@ -17,7 +17,11 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
|
|
|
|
|
-import com.google.common.base.Supplier;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.Iterator;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.Map.Entry;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
|
|
|
import static org.apache.hadoop.fs.CreateFlag.CREATE;
|
|
@@ -45,6 +49,14 @@ import com.google.common.base.Preconditions;
|
|
|
import org.apache.commons.io.IOUtils;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
@@ -68,10 +80,13 @@ import org.apache.hadoop.io.nativeio.NativeIO;
|
|
|
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
-import org.apache.log4j.Level;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Rule;
|
|
|
import org.junit.rules.Timeout;
|
|
|
+import org.slf4j.event.Level;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
+
|
|
|
|
|
|
public abstract class LazyPersistTestCase {
|
|
|
static final byte LAZY_PERSIST_POLICY_ID = (byte) 15;
|
|
@@ -81,16 +96,33 @@ public abstract class LazyPersistTestCase {
|
|
|
GenericTestUtils.setLogLevel(FsDatasetImpl.LOG, Level.DEBUG);
|
|
|
}
|
|
|
|
|
|
+ protected static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(LazyPersistTestCase.class);
|
|
|
protected static final int BLOCK_SIZE = 5 * 1024 * 1024;
|
|
|
protected static final int BUFFER_LENGTH = 4096;
|
|
|
- private static final long HEARTBEAT_INTERVAL_SEC = 1;
|
|
|
- private static final int HEARTBEAT_RECHECK_INTERVAL_MSEC = 500;
|
|
|
- private static final String JMX_RAM_DISK_METRICS_PATTERN = "^RamDisk";
|
|
|
- private static final String JMX_SERVICE_NAME = "DataNode";
|
|
|
protected static final int LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC = 3;
|
|
|
protected static final int LAZY_WRITER_INTERVAL_SEC = 1;
|
|
|
- protected static final Log LOG = LogFactory.getLog(LazyPersistTestCase.class);
|
|
|
protected static final short REPL_FACTOR = 1;
|
|
|
+ private static final String JMX_RAM_DISK_METRICS_PATTERN = "^RamDisk";
|
|
|
+ private static final String JMX_SERVICE_NAME = "DataNode";
|
|
|
+ private static final long HEARTBEAT_INTERVAL_SEC = 1;
|
|
|
+ private static final int HEARTBEAT_RECHECK_INTERVAL_MS = 500;
|
|
|
+ private static final long WAIT_FOR_FBR_MS =
|
|
|
+ TimeUnit.SECONDS.toMillis(10);
|
|
|
+ private static final long WAIT_FOR_STORAGE_TYPES_MS =
|
|
|
+ TimeUnit.SECONDS.toMillis(30);
|
|
|
+ private static final long WAIT_FOR_ASYNC_DELETE_MS =
|
|
|
+ TimeUnit.SECONDS.toMillis(10);
|
|
|
+ private static final long WAIT_FOR_DN_SHUTDOWN_MS =
|
|
|
+ TimeUnit.SECONDS.toMillis(30);
|
|
|
+ private static final long WAIT_FOR_REDUNDANCY_MS =
|
|
|
+ TimeUnit.SECONDS
|
|
|
+ .toMillis(2 * DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT);
|
|
|
+ private static final long WAIT_FOR_LAZY_SCRUBBER_MS =
|
|
|
+ TimeUnit.SECONDS.toMillis(2 * LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC);
|
|
|
+ private static final long WAIT_POLL_INTERVAL_MS = 10;
|
|
|
+ private static final long WAIT_POLL_INTERVAL_LARGE_MS = 20;
|
|
|
+
|
|
|
protected final long osPageSize =
|
|
|
NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize();
|
|
|
|
|
@@ -133,76 +165,79 @@ public abstract class LazyPersistTestCase {
|
|
|
Path path, StorageType storageType)
|
|
|
throws IOException, TimeoutException, InterruptedException {
|
|
|
// Ensure that returned block locations returned are correct!
|
|
|
- LOG.info("Ensure path: " + path + " is on StorageType: " + storageType);
|
|
|
+ LOG.info("Ensure path: {} is on StorageType: {}", path, storageType);
|
|
|
assertThat(fs.exists(path), is(true));
|
|
|
long fileLength = client.getFileInfo(path.toString()).getLen();
|
|
|
|
|
|
- GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
- @Override
|
|
|
- public Boolean get() {
|
|
|
- try {
|
|
|
- LocatedBlocks locatedBlocks =
|
|
|
- client.getLocatedBlocks(path.toString(), 0, fileLength);
|
|
|
- for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
|
|
|
- if (locatedBlock.getStorageTypes()[0] != storageType) {
|
|
|
- return false;
|
|
|
- }
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ LocatedBlocks locatedBlocks =
|
|
|
+ client.getLocatedBlocks(path.toString(), 0, fileLength);
|
|
|
+ for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
|
|
|
+ if (locatedBlock.getStorageTypes()[0] != storageType) {
|
|
|
+ return false;
|
|
|
}
|
|
|
- return true;
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.warn("Exception got in ensureFileReplicasOnStorageType()", ioe);
|
|
|
- return false;
|
|
|
}
|
|
|
+ return true;
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.warn("Exception got in ensureFileReplicasOnStorageType()", ioe);
|
|
|
+ return false;
|
|
|
}
|
|
|
- }, 100, 30 * 1000);
|
|
|
+ }, WAIT_POLL_INTERVAL_MS, WAIT_FOR_STORAGE_TYPES_MS);
|
|
|
|
|
|
return client.getLocatedBlocks(path.toString(), 0, fileLength);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Make sure at least one non-transient volume has a saved copy of the replica.
|
|
|
- * An infinite loop is used to ensure the async lazy persist tasks are completely
|
|
|
- * done before verification. Caller of ensureLazyPersistBlocksAreSaved expects
|
|
|
- * either a successful pass or timeout failure.
|
|
|
+ * Make sure at least one non-transient volume has a saved copy of the
|
|
|
+ * replica. An infinite loop is used to ensure the async lazy persist tasks
|
|
|
+ * are completely done before verification.
|
|
|
+ * Caller of this method expects either a successful pass or timeout failure.
|
|
|
+ *
|
|
|
+ * @param locatedBlocks the collection of blocks and their locations.
|
|
|
+ * @throws IOException for aut-closeable resources.
|
|
|
+ * @throws InterruptedException if the thread is interrupted.
|
|
|
+ * @throws TimeoutException if {@link #WAIT_FOR_STORAGE_TYPES_MS} expires
|
|
|
+ * before we find a persisted copy for each located
|
|
|
+ * block.
|
|
|
*/
|
|
|
protected final void ensureLazyPersistBlocksAreSaved(
|
|
|
- LocatedBlocks locatedBlocks) throws IOException, InterruptedException {
|
|
|
+ final LocatedBlocks locatedBlocks)
|
|
|
+ throws IOException, InterruptedException, TimeoutException {
|
|
|
final String bpid = cluster.getNamesystem().getBlockPoolId();
|
|
|
|
|
|
final Set<Long> persistedBlockIds = new HashSet<Long>();
|
|
|
-
|
|
|
+ // We should find a persisted copy for each located block.
|
|
|
try (FsDatasetSpi.FsVolumeReferences volumes =
|
|
|
cluster.getDataNodes().get(0).getFSDataset().getFsVolumeReferences()) {
|
|
|
- while (persistedBlockIds.size() < locatedBlocks.getLocatedBlocks()
|
|
|
- .size()) {
|
|
|
- // Take 1 second sleep before each verification iteration
|
|
|
- Thread.sleep(1000);
|
|
|
-
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
|
|
|
for (FsVolumeSpi v : volumes) {
|
|
|
if (v.isTransientStorage()) {
|
|
|
continue;
|
|
|
}
|
|
|
-
|
|
|
FsVolumeImpl volume = (FsVolumeImpl) v;
|
|
|
- File lazyPersistDir =
|
|
|
- volume.getBlockPoolSlice(bpid).getLazypersistDir();
|
|
|
-
|
|
|
+ File lazyPersistDir;
|
|
|
+ try {
|
|
|
+ lazyPersistDir =
|
|
|
+ volume.getBlockPoolSlice(bpid).getLazypersistDir();
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
long blockId = lb.getBlock().getBlockId();
|
|
|
File targetDir =
|
|
|
DatanodeUtil.idToBlockDir(lazyPersistDir, blockId);
|
|
|
File blockFile = new File(targetDir, lb.getBlock().getBlockName());
|
|
|
if (blockFile.exists()) {
|
|
|
- // Found a persisted copy for this block and added to the Set
|
|
|
+ // Found a persisted copy for this block and added to the Set.
|
|
|
persistedBlockIds.add(blockId);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- }
|
|
|
+ return (persistedBlockIds.size() ==
|
|
|
+ locatedBlocks.getLocatedBlocks().size());
|
|
|
+ }, WAIT_POLL_INTERVAL_LARGE_MS, WAIT_FOR_STORAGE_TYPES_MS);
|
|
|
}
|
|
|
-
|
|
|
- // We should have found a persisted copy for each located block.
|
|
|
- assertThat(persistedBlockIds.size(), is(locatedBlocks.getLocatedBlocks().size()));
|
|
|
}
|
|
|
|
|
|
protected final void makeRandomTestFile(Path path, long length,
|
|
@@ -271,7 +306,7 @@ public abstract class LazyPersistTestCase {
|
|
|
}
|
|
|
conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL_SEC);
|
|
|
conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
|
|
|
- HEARTBEAT_RECHECK_INTERVAL_MSEC);
|
|
|
+ HEARTBEAT_RECHECK_INTERVAL_MS);
|
|
|
conf.setInt(DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC,
|
|
|
LAZY_WRITER_INTERVAL_SEC);
|
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
|
|
@@ -334,18 +369,18 @@ public abstract class LazyPersistTestCase {
|
|
|
@Override
|
|
|
public void mlock(String identifier,
|
|
|
ByteBuffer mmap, long length) throws IOException {
|
|
|
- LOG.info("LazyPersistTestCase: faking mlock of " + identifier + " bytes.");
|
|
|
+ LOG.info("LazyPersistTestCase: faking mlock of {} bytes.", identifier);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public long getMemlockLimit() {
|
|
|
- LOG.info("LazyPersistTestCase: fake return " + Long.MAX_VALUE);
|
|
|
+ LOG.info("LazyPersistTestCase: fake return {}", Long.MAX_VALUE);
|
|
|
return Long.MAX_VALUE;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public boolean verifyCanMlock() {
|
|
|
- LOG.info("LazyPersistTestCase: fake return " + true);
|
|
|
+ LOG.info("LazyPersistTestCase: fake return {}", true);
|
|
|
return true;
|
|
|
}
|
|
|
});
|
|
@@ -413,8 +448,10 @@ public abstract class LazyPersistTestCase {
|
|
|
|
|
|
public void build() throws IOException {
|
|
|
LazyPersistTestCase.this.startUpCluster(
|
|
|
- numDatanodes, hasTransientStorage, storageTypes, ramDiskReplicaCapacity,
|
|
|
- ramDiskStorageLimit, maxLockedMemory, useScr, useLegacyBlockReaderLocal,
|
|
|
+ numDatanodes, hasTransientStorage, storageTypes,
|
|
|
+ ramDiskReplicaCapacity,
|
|
|
+ ramDiskStorageLimit, maxLockedMemory, useScr,
|
|
|
+ useLegacyBlockReaderLocal,
|
|
|
disableScrubber);
|
|
|
}
|
|
|
|
|
@@ -429,11 +466,44 @@ public abstract class LazyPersistTestCase {
|
|
|
private boolean disableScrubber=false;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Forces a full blockreport on all the datatanodes. The call blocks waiting
|
|
|
+ * for all blockreports to be received by the namenode.
|
|
|
+ *
|
|
|
+ * @throws IOException if an exception is thrown while getting the datanode
|
|
|
+ * descriptors or triggering the blockreports.
|
|
|
+ * @throws InterruptedException if the thread receives an interrupt.
|
|
|
+ * @throws TimeoutException if the reports are not received by
|
|
|
+ * {@link #WAIT_FOR_FBR_MS}.
|
|
|
+ */
|
|
|
protected final void triggerBlockReport()
|
|
|
- throws IOException, InterruptedException {
|
|
|
+ throws InterruptedException, TimeoutException, IOException {
|
|
|
// Trigger block report to NN
|
|
|
- DataNodeTestUtils.triggerBlockReport(cluster.getDataNodes().get(0));
|
|
|
- Thread.sleep(10 * 1000);
|
|
|
+ final Map<DatanodeStorageInfo, Integer> reportCountsBefore =
|
|
|
+ new HashMap<>();
|
|
|
+ final FSNamesystem fsn = cluster.getNamesystem();
|
|
|
+ for (DataNode dn : cluster.getDataNodes()) {
|
|
|
+ final DatanodeDescriptor dnd =
|
|
|
+ NameNodeAdapter.getDatanode(fsn, dn.getDatanodeId());
|
|
|
+ final DatanodeStorageInfo storage = dnd.getStorageInfos()[0];
|
|
|
+ reportCountsBefore.put(storage, storage.getBlockReportCount());
|
|
|
+ DataNodeTestUtils.triggerBlockReport(dn);
|
|
|
+ }
|
|
|
+ // wait for block reports to be received.
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ for (Entry<DatanodeStorageInfo, Integer> reportEntry :
|
|
|
+ reportCountsBefore.entrySet()) {
|
|
|
+ final DatanodeStorageInfo dnStorageInfo = reportEntry.getKey();
|
|
|
+ final int cntBefore = reportEntry.getValue();
|
|
|
+ final int currentCnt = dnStorageInfo.getBlockReportCount();
|
|
|
+ if (cntBefore == currentCnt) {
|
|
|
+ // Same count means no report has been received.
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // If we reach here, then all the block reports have been received.
|
|
|
+ return true;
|
|
|
+ }, WAIT_POLL_INTERVAL_MS, WAIT_FOR_FBR_MS);
|
|
|
}
|
|
|
|
|
|
protected final boolean verifyBlockDeletedFromDir(File dir,
|
|
@@ -445,51 +515,58 @@ public abstract class LazyPersistTestCase {
|
|
|
|
|
|
File blockFile = new File(targetDir, lb.getBlock().getBlockName());
|
|
|
if (blockFile.exists()) {
|
|
|
- LOG.warn("blockFile: " + blockFile.getAbsolutePath() +
|
|
|
- " exists after deletion.");
|
|
|
return false;
|
|
|
}
|
|
|
File metaFile = new File(targetDir,
|
|
|
DatanodeUtil.getMetaName(lb.getBlock().getBlockName(),
|
|
|
lb.getBlock().getGenerationStamp()));
|
|
|
if (metaFile.exists()) {
|
|
|
- LOG.warn("metaFile: " + metaFile.getAbsolutePath() +
|
|
|
- " exists after deletion.");
|
|
|
return false;
|
|
|
}
|
|
|
}
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
- protected final boolean verifyDeletedBlocks(LocatedBlocks locatedBlocks)
|
|
|
- throws IOException, InterruptedException {
|
|
|
+ protected final boolean verifyDeletedBlocks(final LocatedBlocks locatedBlocks)
|
|
|
+ throws Exception {
|
|
|
|
|
|
LOG.info("Verifying replica has no saved copy after deletion.");
|
|
|
triggerBlockReport();
|
|
|
+ final DataNode dn = cluster.getDataNodes().get(0);
|
|
|
|
|
|
- while(
|
|
|
- cluster.getFsDatasetTestUtils(0).getPendingAsyncDeletions()
|
|
|
- > 0L){
|
|
|
- Thread.sleep(1000);
|
|
|
- }
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ for (DataNode dn1 : cluster.getDataNodes()) {
|
|
|
+ if (cluster.getFsDatasetTestUtils(dn1).getPendingAsyncDeletions()
|
|
|
+ > 0) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return true;
|
|
|
+ }, WAIT_POLL_INTERVAL_MS, WAIT_FOR_ASYNC_DELETE_MS);
|
|
|
|
|
|
final String bpid = cluster.getNamesystem().getBlockPoolId();
|
|
|
- final FsDatasetSpi<?> dataset =
|
|
|
- cluster.getDataNodes().get(0).getFSDataset();
|
|
|
-
|
|
|
+ final FsDatasetSpi<?> dataset = dn.getFSDataset();
|
|
|
// Make sure deleted replica does not have a copy on either finalized dir of
|
|
|
- // transient volume or finalized dir of non-transient volume
|
|
|
+ // transient volume or finalized dir of non-transient volume.
|
|
|
+ // We need to wait until the asyn deletion is scheduled.
|
|
|
try (FsDatasetSpi.FsVolumeReferences volumes =
|
|
|
dataset.getFsVolumeReferences()) {
|
|
|
- for (FsVolumeSpi vol : volumes) {
|
|
|
- FsVolumeImpl volume = (FsVolumeImpl) vol;
|
|
|
- File targetDir = (volume.isTransientStorage()) ?
|
|
|
- volume.getBlockPoolSlice(bpid).getFinalizedDir() :
|
|
|
- volume.getBlockPoolSlice(bpid).getLazypersistDir();
|
|
|
- if (verifyBlockDeletedFromDir(targetDir, locatedBlocks) == false) {
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ for (FsVolumeSpi vol : volumes) {
|
|
|
+ FsVolumeImpl volume = (FsVolumeImpl) vol;
|
|
|
+ File targetDir = (volume.isTransientStorage()) ?
|
|
|
+ volume.getBlockPoolSlice(bpid).getFinalizedDir() :
|
|
|
+ volume.getBlockPoolSlice(bpid).getLazypersistDir();
|
|
|
+ if (!verifyBlockDeletedFromDir(targetDir, locatedBlocks)) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return true;
|
|
|
+ } catch (IOException ie) {
|
|
|
return false;
|
|
|
}
|
|
|
- }
|
|
|
+ }, WAIT_POLL_INTERVAL_MS, WAIT_FOR_ASYNC_DELETE_MS);
|
|
|
}
|
|
|
return true;
|
|
|
}
|
|
@@ -530,8 +607,137 @@ public abstract class LazyPersistTestCase {
|
|
|
DFSTestUtil.waitForMetric(jmx, metricName, expectedValue);
|
|
|
}
|
|
|
|
|
|
- protected void triggerEviction(DataNode dn) {
|
|
|
+ protected void triggerEviction(final DataNode dn) {
|
|
|
FsDatasetImpl fsDataset = (FsDatasetImpl) dn.getFSDataset();
|
|
|
fsDataset.evictLazyPersistBlocks(Long.MAX_VALUE); // Run one eviction cycle.
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Shutdown all datanodes in {@link #cluster}. The call blocks for
|
|
|
+ * {@link #WAIT_FOR_DN_SHUTDOWN_MS} until client report has no datanode
|
|
|
+ * labeled as live.
|
|
|
+ *
|
|
|
+ * @throws TimeoutException if {@link #WAIT_FOR_DN_SHUTDOWN_MS} expires with
|
|
|
+ * at least one datanode still alive.
|
|
|
+ * @throws InterruptedException if the thread receives an interrupt.
|
|
|
+ */
|
|
|
+ protected void shutdownDataNodes()
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ cluster.shutdownDataNodes();
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ DatanodeInfo[] info = client.datanodeReport(
|
|
|
+ HdfsConstants.DatanodeReportType.LIVE);
|
|
|
+ return info.length == 0;
|
|
|
+ } catch (IOException e) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }, WAIT_POLL_INTERVAL_LARGE_MS, WAIT_FOR_DN_SHUTDOWN_MS);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Blocks for {@link #WAIT_FOR_REDUNDANCY_MS} waiting for corrupt block count
|
|
|
+ * to reach a certain count.
|
|
|
+ *
|
|
|
+ * @param corruptCnt representing the number of corrupt blocks before
|
|
|
+ * resuming.
|
|
|
+ * @throws TimeoutException if {@link #WAIT_FOR_REDUNDANCY_MS} expires with
|
|
|
+ * corrupt count does not meet the criteria.
|
|
|
+ * @throws InterruptedException if the thread receives an interrupt.
|
|
|
+ */
|
|
|
+ protected void waitForCorruptBlock(final long corruptCnt)
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ // wait for the redundancy monitor to mark the file as corrupt.
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ Iterator<BlockInfo> bInfoIter = cluster.getNameNode()
|
|
|
+ .getNamesystem().getBlockManager().getCorruptReplicaBlockIterator();
|
|
|
+ int count = 0;
|
|
|
+ while (bInfoIter.hasNext()) {
|
|
|
+ bInfoIter.next();
|
|
|
+ count++;
|
|
|
+ }
|
|
|
+ return corruptCnt == count;
|
|
|
+ }, 2 * WAIT_POLL_INTERVAL_LARGE_MS, WAIT_FOR_REDUNDANCY_MS);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Blocks until {@link FSNamesystem#lazyPersistFileScrubber} daemon completes
|
|
|
+ * a full iteration.
|
|
|
+ *
|
|
|
+ * @throws InterruptedException if the thread receives an interrupt.
|
|
|
+ * @throws TimeoutException
|
|
|
+ * {@link FSNamesystem#getLazyPersistFileScrubberTS()}
|
|
|
+ * does not update the timestamp by
|
|
|
+ * {@link #WAIT_FOR_LAZY_SCRUBBER_MS}.
|
|
|
+ */
|
|
|
+ protected void waitForScrubberCycle()
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ // wait for the redundancy monitor to mark the file as corrupt.
|
|
|
+ final FSNamesystem fsn = cluster.getNamesystem();
|
|
|
+ final long lastTimeStamp = fsn.getLazyPersistFileScrubberTS();
|
|
|
+ if (lastTimeStamp == -1) { // scrubber is disabled
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ GenericTestUtils.waitFor(
|
|
|
+ () -> lastTimeStamp != fsn.getLazyPersistFileScrubberTS(),
|
|
|
+ 2 * WAIT_POLL_INTERVAL_LARGE_MS, WAIT_FOR_LAZY_SCRUBBER_MS);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Blocks until {@link BlockManager#RedundancyMonitor} daemon completes
|
|
|
+ * a full iteration.
|
|
|
+ *
|
|
|
+ * @throws InterruptedException if the thread receives an interrupt.
|
|
|
+ * @throws TimeoutException {@link BlockManager#getLastRedundancyMonitorTS()}
|
|
|
+ * does not update the timestamp by
|
|
|
+ * {@link #WAIT_FOR_REDUNDANCY_MS}.
|
|
|
+ */
|
|
|
+ protected void waitForRedundancyMonitorCycle()
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ // wait for the redundancy monitor to mark the file as corrupt.
|
|
|
+ final BlockManager bm = cluster.getNamesystem().getBlockManager();
|
|
|
+ final long lastRedundancyTS =
|
|
|
+ bm.getLastRedundancyMonitorTS();
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(
|
|
|
+ () -> lastRedundancyTS != bm.getLastRedundancyMonitorTS(),
|
|
|
+ 2 * WAIT_POLL_INTERVAL_LARGE_MS, WAIT_FOR_REDUNDANCY_MS);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Blocks until {@link BlockManager#lowRedundancyBlocksCount} reaches a
|
|
|
+ * certain value.
|
|
|
+ *
|
|
|
+ * @throws InterruptedException if the thread receives an interrupt.
|
|
|
+ * @throws TimeoutException {@link BlockManager#getLowRedundancyBlocksCount()}
|
|
|
+ * does not update the count by
|
|
|
+ * {@link #WAIT_FOR_REDUNDANCY_MS}.
|
|
|
+ */
|
|
|
+ protected void waitForLowRedundancyCount(final long cnt)
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ final BlockManager bm = cluster.getNamesystem().getBlockManager();
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() -> cnt == bm.getLowRedundancyBlocksCount(),
|
|
|
+ 2 * WAIT_POLL_INTERVAL_LARGE_MS, WAIT_FOR_REDUNDANCY_MS);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Blocks until the file status changes on the filesystem.
|
|
|
+ *
|
|
|
+ * @param path of the file to be checked.
|
|
|
+ * @param expected whether a file should exist or not.
|
|
|
+ * @throws TimeoutException if the file status does not meet the expected by
|
|
|
+ * {@link #WAIT_FOR_STORAGE_TYPES_MS}.
|
|
|
+ * @throws InterruptedException if the thread receives an interrupt.
|
|
|
+ */
|
|
|
+ protected void waitForFile(final Path path, final boolean expected)
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ return expected == fs.exists(path);
|
|
|
+ } catch (IOException e) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }, WAIT_POLL_INTERVAL_MS, WAIT_FOR_STORAGE_TYPES_MS);
|
|
|
+ }
|
|
|
}
|