HDFS-13947. Review of DirectoryScanner Class. Contributed by BELUGA BEHR.

This commit is contained in:
Inigo Goiri 2018-10-03 11:19:57 -07:00
parent 7051bd78b1
commit 1dc0adfac0
6 changed files with 487 additions and 436 deletions

View File

@ -709,7 +709,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY = DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY =
"dfs.datanode.directoryscan.throttle.limit.ms.per.sec"; "dfs.datanode.directoryscan.throttle.limit.ms.per.sec";
public static final int public static final int
DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT = 1000; DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT = -1;
public static final String DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface"; public static final String DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface";
public static final String DFS_DATANODE_DNS_INTERFACE_DEFAULT = "default"; public static final String DFS_DATANODE_DNS_INTERFACE_DEFAULT = "default";
public static final String DFS_DATANODE_DNS_NAMESERVER_KEY = "dfs.datanode.dns.nameserver"; public static final String DFS_DATANODE_DNS_NAMESERVER_KEY = "dfs.datanode.dns.nameserver";

View File

@ -1086,7 +1086,7 @@ public class DataNode extends ReconfigurableBase
reason = "verifcation is not supported by SimulatedFSDataset"; reason = "verifcation is not supported by SimulatedFSDataset";
} }
if (reason == null) { if (reason == null) {
directoryScanner = new DirectoryScanner(this, data, conf); directoryScanner = new DirectoryScanner(data, conf);
directoryScanner.start(); directoryScanner.start();
} else { } else {
LOG.info("Periodic Directory Tree Verification scan " + LOG.info("Periodic Directory Tree Verification scan " +

View File

@ -17,17 +17,19 @@
*/ */
package org.apache.hadoop.hdfs.server.datanode; package org.apache.hadoop.hdfs.server.datanode;
import com.google.common.annotations.VisibleForTesting;
import java.io.File; import java.io.File;
import java.io.FilenameFilter; import java.io.FilenameFilter;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Set;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
@ -36,23 +38,27 @@ import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang3.time.FastDateFormat; import org.apache.commons.collections.CollectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.AutoCloseableLock;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
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.server.datanode.fsdataset.FsVolumeSpi.ScanInfo; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
import org.apache.hadoop.util.AutoCloseableLock;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.StopWatch; import org.apache.hadoop.util.StopWatch;
import org.apache.hadoop.util.Time; import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
/** /**
* Periodically scans the data directories for block and block metadata files. * Periodically scans the data directories for block and block metadata files.
@ -62,48 +68,48 @@ import org.apache.hadoop.util.Time;
public class DirectoryScanner implements Runnable { public class DirectoryScanner implements Runnable {
private static final Logger LOG = private static final Logger LOG =
LoggerFactory.getLogger(DirectoryScanner.class); LoggerFactory.getLogger(DirectoryScanner.class);
private static final int MILLIS_PER_SECOND = 1000;
private static final String START_MESSAGE = private static final int DEFAULT_MAP_SIZE = 32768;
"Periodic Directory Tree Verification scan"
+ " starting at %s with interval of %dms";
private static final String START_MESSAGE_WITH_THROTTLE = START_MESSAGE
+ " and throttle limit of %dms/s";
private final FsDatasetSpi<?> dataset; private final FsDatasetSpi<?> dataset;
private final ExecutorService reportCompileThreadPool; private final ExecutorService reportCompileThreadPool;
private final ScheduledExecutorService masterThread; private final ScheduledExecutorService masterThread;
private final long scanPeriodMsecs; private final long scanPeriodMsecs;
private final int throttleLimitMsPerSec; private final long throttleLimitMsPerSec;
private volatile boolean shouldRun = false; private final AtomicBoolean shouldRun = new AtomicBoolean();
private boolean retainDiffs = false; private boolean retainDiffs = false;
private final DataNode datanode;
/** /**
* Total combined wall clock time (in milliseconds) spent by the report * Total combined wall clock time (in milliseconds) spent by the report
* compiler threads executing. Used for testing purposes. * compiler threads executing. Used for testing purposes.
*/ */
@VisibleForTesting @VisibleForTesting
final AtomicLong timeRunningMs = new AtomicLong(0L); final AtomicLong timeRunningMs = new AtomicLong(0L);
/** /**
* Total combined wall clock time (in milliseconds) spent by the report * Total combined wall clock time (in milliseconds) spent by the report
* compiler threads blocked by the throttle. Used for testing purposes. * compiler threads blocked by the throttle. Used for testing purposes.
*/ */
@VisibleForTesting @VisibleForTesting
final AtomicLong timeWaitingMs = new AtomicLong(0L); final AtomicLong timeWaitingMs = new AtomicLong(0L);
/** /**
* The complete list of block differences indexed by block pool ID. * The complete list of block differences indexed by block pool ID.
*/ */
@VisibleForTesting @VisibleForTesting
final ScanInfoPerBlockPool diffs = new ScanInfoPerBlockPool(); final BlockPoolReport diffs = new BlockPoolReport();
/** /**
* Statistics about the block differences in each blockpool, indexed by * Statistics about the block differences in each blockpool, indexed by block
* block pool ID. * pool ID.
*/ */
@VisibleForTesting @VisibleForTesting
final Map<String, Stats> stats = new HashMap<String, Stats>(); final Map<String, Stats> stats;
/** /**
* Allow retaining diffs for unit test and analysis. Defaults to false (off) * Allow retaining diffs for unit test and analysis. Defaults to false (off).
*
* @param b whether to retain diffs * @param b whether to retain diffs
*/ */
@VisibleForTesting @VisibleForTesting
@ -123,92 +129,157 @@ public class DirectoryScanner implements Runnable {
long missingMemoryBlocks = 0; long missingMemoryBlocks = 0;
long mismatchBlocks = 0; long mismatchBlocks = 0;
long duplicateBlocks = 0; long duplicateBlocks = 0;
/** /**
* Create a new Stats object for the given blockpool ID. * Create a new Stats object for the given blockpool ID.
*
* @param bpid blockpool ID * @param bpid blockpool ID
*/ */
public Stats(String bpid) { public Stats(String bpid) {
this.bpid = bpid; this.bpid = bpid;
} }
@Override @Override
public String toString() { public String toString() {
return "BlockPool " + bpid return "BlockPool " + bpid + " Total blocks: " + totalBlocks
+ " Total blocks: " + totalBlocks + ", missing metadata files:" + ", missing metadata files: " + missingMetaFile
+ missingMetaFile + ", missing block files:" + missingBlockFile + ", missing block files: " + missingBlockFile
+ ", missing blocks in memory:" + missingMemoryBlocks + ", missing blocks in memory: " + missingMemoryBlocks
+ ", mismatched blocks:" + mismatchBlocks; + ", mismatched blocks: " + mismatchBlocks;
} }
} }
/** /**
* Helper class for compiling block info reports from report compiler threads. * Helper class for compiling block info reports from report compiler threads.
* Contains a volume, a set of block pool IDs, and a collection of ScanInfo
* objects. If a block pool exists but has no ScanInfo objects associated with
* it, there will be no mapping for that particular block pool.
*/ */
static class ScanInfoPerBlockPool extends @VisibleForTesting
HashMap<String, LinkedList<ScanInfo>> { public static class ScanInfoVolumeReport {
@SuppressWarnings("unused")
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
private final FsVolumeSpi volume;
private final BlockPoolReport blockPoolReport;
/** /**
* Create a new info list. * Create a new info list.
*
* @param volume
*/ */
ScanInfoPerBlockPool() {super();} ScanInfoVolumeReport(final FsVolumeSpi volume) {
this.volume = volume;
this.blockPoolReport = new BlockPoolReport();
}
/** /**
* Create a new info list initialized to the given expected size. * Create a new info list initialized to the given expected size.
* See {@link java.util.HashMap#HashMap(int)}.
* *
* @param sz initial expected size * @param sz initial expected size
*/ */
ScanInfoPerBlockPool(int sz) {super(sz);} ScanInfoVolumeReport(final FsVolumeSpi volume,
final Collection<String> blockPools) {
/** this.volume = volume;
* Merges {@code that} ScanInfoPerBlockPool into this one this.blockPoolReport = new BlockPoolReport(blockPools);
*
* @param that ScanInfoPerBlockPool to merge
*/
public void addAll(ScanInfoPerBlockPool that) {
if (that == null) return;
for (Entry<String, LinkedList<ScanInfo>> entry : that.entrySet()) {
String bpid = entry.getKey();
LinkedList<ScanInfo> list = entry.getValue();
if (this.containsKey(bpid)) {
//merge that per-bpid linked list with this one
this.get(bpid).addAll(list);
} else {
//add that new bpid and its linked list to this
this.put(bpid, list);
}
}
} }
/** public void addAll(final String bpid,
* Convert all the LinkedList values in this ScanInfoPerBlockPool map final Collection<ScanInfo> scanInfos) {
* into sorted arrays, and return a new map of these arrays per blockpool this.blockPoolReport.addAll(bpid, scanInfos);
* }
* @return a map of ScanInfo arrays per blockpool
*/ public Set<String> getBlockPoolIds() {
public Map<String, ScanInfo[]> toSortedArrays() { return this.blockPoolReport.getBlockPoolIds();
Map<String, ScanInfo[]> result = }
new HashMap<String, ScanInfo[]>(this.size());
public List<ScanInfo> getScanInfo(final String bpid) {
for (Entry<String, LinkedList<ScanInfo>> entry : this.entrySet()) { return this.blockPoolReport.getScanInfo(bpid);
String bpid = entry.getKey(); }
LinkedList<ScanInfo> list = entry.getValue();
public FsVolumeSpi getVolume() {
// convert list to array return volume;
ScanInfo[] record = list.toArray(new ScanInfo[list.size()]); }
// Sort array based on blockId
Arrays.sort(record); @Override
result.put(bpid, record); public String toString() {
} return "ScanInfoVolumeReport [volume=" + volume + ", blockPoolReport="
return result; + blockPoolReport + "]";
} }
} }
/**
* Helper class for compiling block info reports per block pool.
*/
@VisibleForTesting
public static class BlockPoolReport {
@SuppressWarnings("unused")
private static final long serialVersionUID = 1L;
private final Set<String> blockPools;
private final ListMultimap<String, ScanInfo> map;
/**
* Create a block pool report.
*
* @param volume
*/
BlockPoolReport() {
this.blockPools = new HashSet<>(2);
this.map = ArrayListMultimap.create(2, DEFAULT_MAP_SIZE);
}
/**
* Create a new block pool report initialized to the given expected size.
*
* @param blockPools initial list of known block pools
*/
BlockPoolReport(final Collection<String> blockPools) {
this.blockPools = new HashSet<>(blockPools);
this.map = ArrayListMultimap.create(blockPools.size(), DEFAULT_MAP_SIZE);
}
public void addAll(final String bpid,
final Collection<ScanInfo> scanInfos) {
this.blockPools.add(bpid);
this.map.putAll(bpid, scanInfos);
}
public void sortBlocks() {
for (final String bpid : this.map.keySet()) {
final List<ScanInfo> list = this.map.get(bpid);
// Sort array based on blockId
Collections.sort(list);
}
}
public Set<String> getBlockPoolIds() {
return Collections.unmodifiableSet(this.blockPools);
}
public List<ScanInfo> getScanInfo(final String bpid) {
return this.map.get(bpid);
}
public Collection<Map.Entry<String, ScanInfo>> getEntries() {
return Collections.unmodifiableCollection(this.map.entries());
}
public void clear() {
this.map.clear();
this.blockPools.clear();
}
@Override
public String toString() {
return "BlockPoolReport [blockPools=" + blockPools + ", map=" + map + "]";
}
}
/** /**
* Create a new directory scanner, but don't cycle it running yet. * Create a new directory scanner, but don't cycle it running yet.
@ -217,75 +288,58 @@ public class DirectoryScanner implements Runnable {
* @param dataset the dataset to scan * @param dataset the dataset to scan
* @param conf the Configuration object * @param conf the Configuration object
*/ */
public DirectoryScanner(DataNode datanode, FsDatasetSpi<?> dataset, public DirectoryScanner(FsDatasetSpi<?> dataset, Configuration conf) {
Configuration conf) {
this.datanode = datanode;
this.dataset = dataset; this.dataset = dataset;
this.stats = new HashMap<>(DEFAULT_MAP_SIZE);
int interval = (int) conf.getTimeDuration( int interval = (int) conf.getTimeDuration(
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY,
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT, DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT,
TimeUnit.SECONDS); TimeUnit.SECONDS);
scanPeriodMsecs = interval * MILLIS_PER_SECOND; //msec
int throttle = scanPeriodMsecs = TimeUnit.SECONDS.toMillis(interval);
conf.getInt(
int throttle = conf.getInt(
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT);
if (throttle >= TimeUnit.SECONDS.toMillis(1)) {
LOG.warn(
"{} set to value above 1000 ms/sec. Assuming default value of {}",
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY, DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT); DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT);
throttle =
if ((throttle > MILLIS_PER_SECOND) || (throttle <= 0)) {
if (throttle > MILLIS_PER_SECOND) {
LOG.error(
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY
+ " set to value above 1000 ms/sec. Assuming default value of " +
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT);
} else {
LOG.error(
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY
+ " set to value below 1 ms/sec. Assuming default value of " +
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT);
}
throttleLimitMsPerSec =
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT; DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT;
} else {
throttleLimitMsPerSec = throttle;
} }
int threads = throttleLimitMsPerSec = throttle;
conf.getInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_DEFAULT);
reportCompileThreadPool = Executors.newFixedThreadPool(threads, int threads =
new Daemon.DaemonFactory()); conf.getInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
masterThread = new ScheduledThreadPoolExecutor(1, DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_DEFAULT);
new Daemon.DaemonFactory());
reportCompileThreadPool =
Executors.newFixedThreadPool(threads, new Daemon.DaemonFactory());
masterThread =
new ScheduledThreadPoolExecutor(1, new Daemon.DaemonFactory());
} }
/** /**
* Start the scanner. The scanner will run every * Start the scanner. The scanner will run every
* {@link DFSConfigKeys#DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY} seconds. * {@link DFSConfigKeys#DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY} seconds.
*/ */
void start() { void start() {
shouldRun = true; shouldRun.set(true);
long offset = ThreadLocalRandom.current().nextInt( long firstScanTime = ThreadLocalRandom.current().nextLong(scanPeriodMsecs);
(int) (scanPeriodMsecs/MILLIS_PER_SECOND)) * MILLIS_PER_SECOND; //msec
long firstScanTime = Time.now() + offset;
String logMsg;
if (throttleLimitMsPerSec < MILLIS_PER_SECOND) { LOG.info(
logMsg = String.format(START_MESSAGE_WITH_THROTTLE, "Periodic Directory Tree Verification scan starting in {}ms with interval of {}ms and throttle limit of {}ms/s",
FastDateFormat.getInstance().format(firstScanTime), scanPeriodMsecs, firstScanTime, scanPeriodMsecs, throttleLimitMsPerSec);
throttleLimitMsPerSec);
} else {
logMsg = String.format(START_MESSAGE,
FastDateFormat.getInstance().format(firstScanTime), scanPeriodMsecs);
}
LOG.info(logMsg); masterThread.scheduleAtFixedRate(this, firstScanTime, scanPeriodMsecs,
masterThread.scheduleAtFixedRate(this, offset, scanPeriodMsecs, TimeUnit.MILLISECONDS);
TimeUnit.MILLISECONDS);
} }
/** /**
* Return whether the scanner has been started. * Return whether the scanner has been started.
* *
@ -293,7 +347,7 @@ public class DirectoryScanner implements Runnable {
*/ */
@VisibleForTesting @VisibleForTesting
boolean getRunStatus() { boolean getRunStatus() {
return shouldRun; return shouldRun.get();
} }
/** /**
@ -305,67 +359,69 @@ public class DirectoryScanner implements Runnable {
} }
/** /**
* Main program loop for DirectoryScanner. Runs {@link reconcile()} * Main program loop for DirectoryScanner. Runs {@link reconcile()} and
* and handles any exceptions. * handles any exceptions.
*/ */
@Override @Override
public void run() { public void run() {
if (!shouldRun.get()) {
// shutdown has been activated
LOG.warn(
"This cycle terminating immediately because 'shouldRun' has been deactivated");
return;
}
try { try {
if (!shouldRun) { reconcile();
//shutdown has been activated
LOG.warn("this cycle terminating immediately because 'shouldRun' has been deactivated");
return;
}
//We're are okay to run - do it
reconcile();
} catch (Exception e) { } catch (Exception e) {
//Log and continue - allows Executor to run again next cycle // Log and continue - allows Executor to run again next cycle
LOG.error("Exception during DirectoryScanner execution - will continue next cycle", e); LOG.error(
"Exception during DirectoryScanner execution - will continue next cycle",
e);
} catch (Error er) { } catch (Error er) {
//Non-recoverable error - re-throw after logging the problem // Non-recoverable error - re-throw after logging the problem
LOG.error("System Error during DirectoryScanner execution - permanently terminating periodic scanner", er); LOG.error(
"System Error during DirectoryScanner execution - permanently terminating periodic scanner",
er);
throw er; throw er;
} }
} }
/** /**
* Stops the directory scanner. This method will wait for 1 minute for the * Stops the directory scanner. This method will wait for 1 minute for the
* main thread to exit and an additional 1 minute for the report compilation * main thread to exit and an additional 1 minute for the report compilation
* threads to exit. If a thread does not exit in that time period, it is * threads to exit. If a thread does not exit in that time period, it is left
* left running, and an error is logged. * running, and an error is logged.
*/ */
void shutdown() { void shutdown() {
if (!shouldRun) { LOG.info("Shutdown has been called");
LOG.warn("DirectoryScanner: shutdown has been called, but periodic scanner not started"); if (!shouldRun.getAndSet(false)) {
} else { LOG.warn("Shutdown has been called, but periodic scanner not started");
LOG.warn("DirectoryScanner: shutdown has been called"); }
if (masterThread != null) {
masterThread.shutdown();
} }
shouldRun = false;
if (masterThread != null) masterThread.shutdown();
if (reportCompileThreadPool != null) { if (reportCompileThreadPool != null) {
reportCompileThreadPool.shutdownNow(); reportCompileThreadPool.shutdownNow();
} }
if (masterThread != null) { if (masterThread != null) {
try { try {
masterThread.awaitTermination(1, TimeUnit.MINUTES); masterThread.awaitTermination(1, TimeUnit.MINUTES);
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.error("interrupted while waiting for masterThread to " + LOG.error(
"terminate", e); "interrupted while waiting for masterThread to " + "terminate", e);
} }
} }
if (reportCompileThreadPool != null) { if (reportCompileThreadPool != null) {
try { try {
reportCompileThreadPool.awaitTermination(1, TimeUnit.MINUTES); reportCompileThreadPool.awaitTermination(1, TimeUnit.MINUTES);
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.error("interrupted while waiting for reportCompileThreadPool to " + LOG.error("interrupted while waiting for reportCompileThreadPool to "
"terminate", e); + "terminate", e);
} }
} }
if (!retainDiffs) clear(); if (!retainDiffs) {
clear();
}
} }
/** /**
@ -374,45 +430,54 @@ public class DirectoryScanner implements Runnable {
@VisibleForTesting @VisibleForTesting
public void reconcile() throws IOException { public void reconcile() throws IOException {
scan(); scan();
for (Entry<String, LinkedList<ScanInfo>> entry : diffs.entrySet()) {
String bpid = entry.getKey(); for (final Map.Entry<String, ScanInfo> entry : diffs.getEntries()) {
LinkedList<ScanInfo> diff = entry.getValue(); dataset.checkAndUpdate(entry.getKey(), entry.getValue());
}
for (ScanInfo info : diff) {
dataset.checkAndUpdate(bpid, info); if (!retainDiffs) {
} clear();
} }
if (!retainDiffs) clear();
} }
/** /**
* Scan for the differences between disk and in-memory blocks * Scan for the differences between disk and in-memory blocks Scan only the
* Scan only the "finalized blocks" lists of both disk and memory. * "finalized blocks" lists of both disk and memory.
*/ */
private void scan() { private void scan() {
BlockPoolReport blockPoolReport = new BlockPoolReport();
clear(); clear();
Map<String, ScanInfo[]> diskReport = getDiskReport();
Collection<ScanInfoVolumeReport> volumeReports = getVolumeReports();
for (ScanInfoVolumeReport volumeReport : volumeReports) {
for (String blockPoolId : volumeReport.getBlockPoolIds()) {
List<ScanInfo> scanInfos = volumeReport.getScanInfo(blockPoolId);
blockPoolReport.addAll(blockPoolId, scanInfos);
}
}
// Pre-sort the reports outside of the lock
blockPoolReport.sortBlocks();
// Hold FSDataset lock to prevent further changes to the block map // Hold FSDataset lock to prevent further changes to the block map
try(AutoCloseableLock lock = dataset.acquireDatasetLock()) { try (AutoCloseableLock lock = dataset.acquireDatasetLock()) {
for (Entry<String, ScanInfo[]> entry : diskReport.entrySet()) { for (final String bpid : blockPoolReport.getBlockPoolIds()) {
String bpid = entry.getKey(); List<ScanInfo> blockpoolReport = blockPoolReport.getScanInfo(bpid);
ScanInfo[] blockpoolReport = entry.getValue();
Stats statsRecord = new Stats(bpid); Stats statsRecord = new Stats(bpid);
stats.put(bpid, statsRecord); stats.put(bpid, statsRecord);
LinkedList<ScanInfo> diffRecord = new LinkedList<ScanInfo>(); Collection<ScanInfo> diffRecord = new ArrayList<>();
diffs.put(bpid, diffRecord);
statsRecord.totalBlocks = blockpoolReport.size();
statsRecord.totalBlocks = blockpoolReport.length;
final List<ReplicaInfo> bl = dataset.getFinalizedBlocks(bpid); final List<ReplicaInfo> bl = dataset.getFinalizedBlocks(bpid);
Collections.sort(bl); // Sort based on blockId Collections.sort(bl); // Sort based on blockId
int d = 0; // index for blockpoolReport int d = 0; // index for blockpoolReport
int m = 0; // index for memReprot int m = 0; // index for memReprot
while (m < bl.size() && d < blockpoolReport.length) { while (m < bl.size() && d < blockpoolReport.size()) {
ReplicaInfo memBlock = bl.get(m); ReplicaInfo memBlock = bl.get(m);
ScanInfo info = blockpoolReport[d]; ScanInfo info = blockpoolReport.get(d);
if (info.getBlockId() < memBlock.getBlockId()) { if (info.getBlockId() < memBlock.getBlockId()) {
if (!dataset.isDeletingBlock(bpid, info.getBlockId())) { if (!dataset.isDeletingBlock(bpid, info.getBlockId())) {
// Block is missing in memory // Block is missing in memory
@ -424,15 +489,15 @@ public class DirectoryScanner implements Runnable {
} }
if (info.getBlockId() > memBlock.getBlockId()) { if (info.getBlockId() > memBlock.getBlockId()) {
// Block is missing on the disk // Block is missing on the disk
addDifference(diffRecord, statsRecord, addDifference(diffRecord, statsRecord, memBlock.getBlockId(),
memBlock.getBlockId(), info.getVolume()); info.getVolume());
m++; m++;
continue; continue;
} }
// Block file and/or metadata file exists on the disk // Block file and/or metadata file exists on the disk
// Block exists in memory // Block exists in memory
if (info.getVolume().getStorageType() != StorageType.PROVIDED && if (info.getVolume().getStorageType() != StorageType.PROVIDED
info.getBlockFile() == null) { && info.getBlockFile() == null) {
// Block metadata file exits and block file is missing // Block metadata file exits and block file is missing
addDifference(diffRecord, statsRecord, info); addDifference(diffRecord, statsRecord, info);
} else if (info.getGenStamp() != memBlock.getGenerationStamp() } else if (info.getGenStamp() != memBlock.getGenerationStamp()
@ -442,16 +507,16 @@ public class DirectoryScanner implements Runnable {
statsRecord.mismatchBlocks++; statsRecord.mismatchBlocks++;
addDifference(diffRecord, statsRecord, info); addDifference(diffRecord, statsRecord, info);
} else if (memBlock.compareWith(info) != 0) { } else if (memBlock.compareWith(info) != 0) {
// volumeMap record and on-disk files don't match. // volumeMap record and on-disk files do not match.
statsRecord.duplicateBlocks++; statsRecord.duplicateBlocks++;
addDifference(diffRecord, statsRecord, info); addDifference(diffRecord, statsRecord, info);
} }
d++; d++;
if (d < blockpoolReport.length) { if (d < blockpoolReport.size()) {
// There may be multiple on-disk records for the same block, don't increment // There may be multiple on-disk records for the same block, do not
// the memory record pointer if so. // increment the memory record pointer if so.
ScanInfo nextInfo = blockpoolReport[d]; ScanInfo nextInfo = blockpoolReport.get(d);
if (nextInfo.getBlockId() != info.getBlockId()) { if (nextInfo.getBlockId() != info.getBlockId()) {
++m; ++m;
} }
@ -461,132 +526,108 @@ public class DirectoryScanner implements Runnable {
} }
while (m < bl.size()) { while (m < bl.size()) {
ReplicaInfo current = bl.get(m++); ReplicaInfo current = bl.get(m++);
addDifference(diffRecord, statsRecord, addDifference(diffRecord, statsRecord, current.getBlockId(),
current.getBlockId(), current.getVolume()); current.getVolume());
} }
while (d < blockpoolReport.length) { while (d < blockpoolReport.size()) {
if (!dataset.isDeletingBlock(bpid, blockpoolReport[d].getBlockId())) { if (!dataset.isDeletingBlock(bpid,
blockpoolReport.get(d).getBlockId())) {
statsRecord.missingMemoryBlocks++; statsRecord.missingMemoryBlocks++;
addDifference(diffRecord, statsRecord, blockpoolReport[d]); addDifference(diffRecord, statsRecord, blockpoolReport.get(d));
} }
d++; d++;
} }
LOG.info(statsRecord.toString()); diffs.addAll(bpid, diffRecord);
} //end for LOG.info("Scan Results: {}", statsRecord);
} //end synchronized }
}
} }
/** /**
* Add the ScanInfo object to the list of differences and adjust the stats * Add the ScanInfo object to the list of differences and adjust the stats
* accordingly. This method is called when a block is found on the disk, * accordingly. This method is called when a block is found on the disk, but
* but the in-memory block is missing or does not match the block on the disk. * the in-memory block is missing or does not match the block on the disk.
* *
* @param diffRecord the list to which to add the info * @param diffRecord the collection to which to add the info
* @param statsRecord the stats to update * @param statsRecord the stats to update
* @param info the differing info * @param info the differing info
*/ */
private void addDifference(LinkedList<ScanInfo> diffRecord, private void addDifference(Collection<ScanInfo> diffRecord, Stats statsRecord,
Stats statsRecord, ScanInfo info) { ScanInfo info) {
statsRecord.missingMetaFile += info.getMetaFile() == null ? 1 : 0; statsRecord.missingMetaFile += info.getMetaFile() == null ? 1 : 0;
statsRecord.missingBlockFile += info.getBlockFile() == null ? 1 : 0; statsRecord.missingBlockFile += info.getBlockFile() == null ? 1 : 0;
diffRecord.add(info); diffRecord.add(info);
} }
/** /**
* Add a new ScanInfo object to the list of differences and adjust the stats * Add a new ScanInfo object to the collection of differences and adjust the
* accordingly. This method is called when a block is not found on the disk. * stats accordingly. This method is called when a block is not found on the
* disk.
* *
* @param diffRecord the list to which to add the info * @param diffRecord the collection to which to add the info
* @param statsRecord the stats to update * @param statsRecord the stats to update
* @param blockId the id of the missing block * @param blockId the id of the missing block
* @param vol the volume that contains the missing block * @param vol the volume that contains the missing block
*/ */
private void addDifference(LinkedList<ScanInfo> diffRecord, private void addDifference(Collection<ScanInfo> diffRecord, Stats statsRecord,
Stats statsRecord, long blockId, long blockId, FsVolumeSpi vol) {
FsVolumeSpi vol) {
statsRecord.missingBlockFile++; statsRecord.missingBlockFile++;
statsRecord.missingMetaFile++; statsRecord.missingMetaFile++;
diffRecord.add(new ScanInfo(blockId, null, null, vol)); diffRecord.add(new ScanInfo(blockId, null, null, vol));
} }
/** /**
* Get the lists of blocks on the disks in the dataset, sorted by blockId. * Get the lists of blocks on the disks in the data set.
* The returned map contains one entry per blockpool, keyed by the blockpool
* ID.
*
* @return a map of sorted arrays of block information
*/ */
@VisibleForTesting @VisibleForTesting
public Map<String, ScanInfo[]> getDiskReport() { public Collection<ScanInfoVolumeReport> getVolumeReports() {
ScanInfoPerBlockPool list = new ScanInfoPerBlockPool(); List<ScanInfoVolumeReport> volReports = new ArrayList<>();
ScanInfoPerBlockPool[] dirReports = null; List<Future<ScanInfoVolumeReport>> compilersInProgress = new ArrayList<>();
// First get list of data directories // First get list of data directories
try (FsDatasetSpi.FsVolumeReferences volumes = try (FsDatasetSpi.FsVolumeReferences volumes =
dataset.getFsVolumeReferences()) { dataset.getFsVolumeReferences()) {
// Use an array since the threads may return out of order and for (final FsVolumeSpi volume : volumes) {
// compilersInProgress#keySet may return out of order as well. // Disable scanning PROVIDED volumes to keep overhead low
dirReports = new ScanInfoPerBlockPool[volumes.size()]; if (volume.getStorageType() != StorageType.PROVIDED) {
ReportCompiler reportCompiler = new ReportCompiler(volume);
Map<Integer, Future<ScanInfoPerBlockPool>> compilersInProgress = Future<ScanInfoVolumeReport> result =
new HashMap<Integer, Future<ScanInfoPerBlockPool>>(); reportCompileThreadPool.submit(reportCompiler);
compilersInProgress.add(result);
for (int i = 0; i < volumes.size(); i++) {
if (volumes.get(i).getStorageType() == StorageType.PROVIDED) {
// Disable scanning PROVIDED volumes to keep overhead low
continue;
} }
ReportCompiler reportCompiler =
new ReportCompiler(datanode, volumes.get(i));
Future<ScanInfoPerBlockPool> result =
reportCompileThreadPool.submit(reportCompiler);
compilersInProgress.put(i, result);
} }
for (Entry<Integer, Future<ScanInfoPerBlockPool>> report : for (Future<ScanInfoVolumeReport> future : compilersInProgress) {
compilersInProgress.entrySet()) {
Integer index = report.getKey();
try { try {
dirReports[index] = report.getValue().get(); final ScanInfoVolumeReport result = future.get();
if (!CollectionUtils.addIgnoreNull(volReports, result)) {
// If our compiler threads were interrupted, give up on this run // This compiler thread were interrupted, give up on this run
if (dirReports[index] == null) { volReports.clear();
dirReports = null;
break; break;
} }
} catch (Exception ex) { } catch (Exception ex) {
FsVolumeSpi fsVolumeSpi = volumes.get(index); LOG.warn("Error compiling report. Continuing.", ex);
LOG.error("Error compiling report for the volume, StorageId: "
+ fsVolumeSpi.getStorageID(), ex);
// Continue scanning the other volumes
} }
} }
} catch (IOException e) { } catch (IOException e) {
LOG.error("Unexpected IOException by closing FsVolumeReference", e); LOG.error("Unexpected IOException by closing FsVolumeReference", e);
} }
if (dirReports != null) {
// Compile consolidated report for all the volumes return volReports;
for (ScanInfoPerBlockPool report : dirReports) {
if(report != null){
list.addAll(report);
}
}
}
return list.toSortedArrays();
} }
/** /**
* The ReportCompiler class encapsulates the process of searching a datanode's * The ReportCompiler class encapsulates the process of searching a datanode's
* disks for block information. It operates by performing a DFS of the * disks for block information. It operates by performing a DFS of the volume
* volume to discover block information. * to discover block information.
* *
* When the ReportCompiler discovers block information, it create a new * When the ReportCompiler discovers block information, it create a new
* ScanInfo object for it and adds that object to its report list. The report * ScanInfo object for it and adds that object to its report list. The report
* list is returned by the {@link #call()} method. * list is returned by the {@link #call()} method.
*/ */
public class ReportCompiler implements Callable<ScanInfoPerBlockPool> { public class ReportCompiler implements Callable<ScanInfoVolumeReport> {
private final FsVolumeSpi volume; private final FsVolumeSpi volume;
private final DataNode datanode;
// Variable for tracking time spent running for throttling purposes // Variable for tracking time spent running for throttling purposes
private final StopWatch throttleTimer = new StopWatch(); private final StopWatch throttleTimer = new StopWatch();
// Variable for tracking time spent running and waiting for testing // Variable for tracking time spent running and waiting for testing
@ -594,13 +635,11 @@ public class DirectoryScanner implements Runnable {
private final StopWatch perfTimer = new StopWatch(); private final StopWatch perfTimer = new StopWatch();
/** /**
* Create a report compiler for the given volume on the given datanode. * Create a report compiler for the given volume.
* *
* @param datanode the target datanode
* @param volume the target volume * @param volume the target volume
*/ */
public ReportCompiler(DataNode datanode, FsVolumeSpi volume) { public ReportCompiler(FsVolumeSpi volume) {
this.datanode = datanode;
this.volume = volume; this.volume = volume;
} }
@ -608,12 +647,13 @@ public class DirectoryScanner implements Runnable {
* Run this report compiler thread. * Run this report compiler thread.
* *
* @return the block info report list * @return the block info report list
* @throws IOException if the block pool isn't found * @throws IOException if the block pool is not found
*/ */
@Override @Override
public ScanInfoPerBlockPool call() throws IOException { public ScanInfoVolumeReport call() throws IOException {
String[] bpList = volume.getBlockPoolList(); String[] bpList = volume.getBlockPoolList();
ScanInfoPerBlockPool result = new ScanInfoPerBlockPool(bpList.length); ScanInfoVolumeReport result =
new ScanInfoVolumeReport(volume, Arrays.asList(bpList));
perfTimer.start(); perfTimer.start();
throttleTimer.start(); throttleTimer.start();
for (String bpid : bpList) { for (String bpid : bpList) {
@ -623,33 +663,45 @@ public class DirectoryScanner implements Runnable {
throttleTimer.reset().start(); throttleTimer.reset().start();
try { try {
result.put(bpid, volume.compileReport(bpid, report, this)); // ScanInfos are added directly to 'report' list
volume.compileReport(bpid, report, this);
result.addAll(bpid, report);
} catch (InterruptedException ex) { } catch (InterruptedException ex) {
// Exit quickly and flag the scanner to do the same // Exit quickly and flag the scanner to do the same
result = null; result = null;
break; break;
} }
} }
LOG.trace("Scanner volume report: {}", result);
return result; return result;
} }
/** /**
* Called by the thread before each potential disk scan so that a pause * Called by the thread before each potential disk scan so that a pause can
* can be optionally inserted to limit the number of scans per second. * be optionally inserted to limit the number of scans per second. The limit
* The limit is controlled by * is controlled by
* {@link DFSConfigKeys#DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY}. * {@link DFSConfigKeys#DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY}.
*/ */
public void throttle() throws InterruptedException { public void throttle() throws InterruptedException {
accumulateTimeRunning(); accumulateTimeRunning();
if ((throttleLimitMsPerSec < 1000) && if (throttleLimitMsPerSec > 0L) {
(throttleTimer.now(TimeUnit.MILLISECONDS) > throttleLimitMsPerSec)) { final long runningTime = throttleTimer.now(TimeUnit.MILLISECONDS);
if (runningTime >= throttleLimitMsPerSec) {
Thread.sleep(MILLIS_PER_SECOND - throttleLimitMsPerSec); final long sleepTime;
throttleTimer.reset().start(); if (runningTime >= 1000L) {
LOG.warn("Unable to throttle within the second. Blocking for 1s.");
sleepTime = 1000L;
} else {
// Sleep for the expected time plus any time processing ran over
final long overTime = runningTime - throttleLimitMsPerSec;
sleepTime = (1000L - throttleLimitMsPerSec) + overTime;
}
Thread.sleep(sleepTime);
throttleTimer.reset().start();
}
accumulateTimeWaiting();
} }
accumulateTimeWaiting();
} }
/** /**
@ -679,4 +731,5 @@ public class DirectoryScanner implements Runnable {
|| name.startsWith(Block.BLOCK_FILE_PREFIX); || name.startsWith(Block.BLOCK_FILE_PREFIX);
} }
} }
} }

View File

@ -33,6 +33,7 @@ import java.net.URI;
import java.nio.channels.ClosedChannelException; import java.nio.channels.ClosedChannelException;
import java.nio.channels.FileChannel; import java.nio.channels.FileChannel;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator; import java.util.Iterator;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
@ -44,26 +45,23 @@ import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DF; import org.apache.hadoop.fs.DF;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.DataNodeVolumeMetrics;
import org.apache.hadoop.util.AutoCloseableLock;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler; import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.DataNodeVolumeMetrics;
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.FsDatasetSpi.FsVolumeReferences; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
@ -73,14 +71,17 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.AutoCloseableLock;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** /**
* Tests {@link DirectoryScanner} handling of differences * Tests {@link DirectoryScanner} handling of differences between blocks on the
* between blocks on the disk and block in memory. * disk and block in memory.
*/ */
public class TestDirectoryScanner { public class TestDirectoryScanner {
private static final Logger LOG = private static final Logger LOG =
@ -102,7 +103,7 @@ public class TestDirectoryScanner {
CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1); CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
CONF.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, CONF.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
getMemlockLimit(Long.MAX_VALUE)); getMemlockLimit(Long.MAX_VALUE));
} }
@Before @Before
@ -110,21 +111,20 @@ public class TestDirectoryScanner {
LazyPersistTestCase.initCacheManipulator(); LazyPersistTestCase.initCacheManipulator();
} }
/** create a file with a length of <code>fileLen</code> */ /** create a file with a length of <code>fileLen</code>. */
private List<LocatedBlock> createFile(String fileNamePrefix, private List<LocatedBlock> createFile(String fileNamePrefix, long fileLen,
long fileLen, boolean isLazyPersist) throws IOException {
boolean isLazyPersist) throws IOException {
FileSystem fs = cluster.getFileSystem(); FileSystem fs = cluster.getFileSystem();
Path filePath = new Path("/" + fileNamePrefix + ".dat"); Path filePath = new Path("/" + fileNamePrefix + ".dat");
DFSTestUtil.createFile( DFSTestUtil.createFile(fs, filePath, isLazyPersist, 1024, fileLen,
fs, filePath, isLazyPersist, 1024, fileLen,
BLOCK_LENGTH, (short) 1, r.nextLong(), false); BLOCK_LENGTH, (short) 1, r.nextLong(), false);
return client.getLocatedBlocks(filePath.toString(), 0, fileLen).getLocatedBlocks(); return client.getLocatedBlocks(filePath.toString(), 0, fileLen)
.getLocatedBlocks();
} }
/** Truncate a block file */ /** Truncate a block file. */
private long truncateBlockFile() throws IOException { private long truncateBlockFile() throws IOException {
try(AutoCloseableLock lock = fds.acquireDatasetLock()) { try (AutoCloseableLock lock = fds.acquireDatasetLock()) {
for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) { for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
File f = new File(b.getBlockURI()); File f = new File(b.getBlockURI());
File mf = new File(b.getMetadataURI()); File mf = new File(b.getMetadataURI());
@ -149,7 +149,7 @@ public class TestDirectoryScanner {
/** Delete a block file */ /** Delete a block file */
private long deleteBlockFile() { private long deleteBlockFile() {
try(AutoCloseableLock lock = fds.acquireDatasetLock()) { try (AutoCloseableLock lock = fds.acquireDatasetLock()) {
for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) { for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
File f = new File(b.getBlockURI()); File f = new File(b.getBlockURI());
File mf = new File(b.getMetadataURI()); File mf = new File(b.getMetadataURI());
@ -165,7 +165,7 @@ public class TestDirectoryScanner {
/** Delete block meta file */ /** Delete block meta file */
private long deleteMetaFile() { private long deleteMetaFile() {
try(AutoCloseableLock lock = fds.acquireDatasetLock()) { try (AutoCloseableLock lock = fds.acquireDatasetLock()) {
for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) { for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
// Delete a metadata file // Delete a metadata file
if (b.metadataExists() && b.deleteMetadata()) { if (b.metadataExists() && b.deleteMetadata()) {
@ -179,11 +179,12 @@ public class TestDirectoryScanner {
/** /**
* Duplicate the given block on all volumes. * Duplicate the given block on all volumes.
*
* @param blockId * @param blockId
* @throws IOException * @throws IOException
*/ */
private void duplicateBlock(long blockId) throws IOException { private void duplicateBlock(long blockId) throws IOException {
try(AutoCloseableLock lock = fds.acquireDatasetLock()) { try (AutoCloseableLock lock = fds.acquireDatasetLock()) {
ReplicaInfo b = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId); ReplicaInfo b = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
try (FsDatasetSpi.FsVolumeReferences volumes = try (FsDatasetSpi.FsVolumeReferences volumes =
fds.getFsVolumeReferences()) { fds.getFsVolumeReferences()) {
@ -199,16 +200,14 @@ public class TestDirectoryScanner {
URI destRoot = v.getStorageLocation().getUri(); URI destRoot = v.getStorageLocation().getUri();
String relativeBlockPath = String relativeBlockPath =
sourceRoot.relativize(sourceBlock.toURI()) sourceRoot.relativize(sourceBlock.toURI()).getPath();
.getPath();
String relativeMetaPath = String relativeMetaPath =
sourceRoot.relativize(sourceMeta.toURI()) sourceRoot.relativize(sourceMeta.toURI()).getPath();
.getPath();
File destBlock = new File(new File(destRoot).toString(), File destBlock =
relativeBlockPath); new File(new File(destRoot).toString(), relativeBlockPath);
File destMeta = new File(new File(destRoot).toString(), File destMeta =
relativeMetaPath); new File(new File(destRoot).toString(), relativeMetaPath);
destBlock.getParentFile().mkdirs(); destBlock.getParentFile().mkdirs();
FileUtils.copyFile(sourceBlock, destBlock); FileUtils.copyFile(sourceBlock, destBlock);
@ -223,7 +222,7 @@ public class TestDirectoryScanner {
} }
} }
/** Get a random blockId that is not used already */ /** Get a random blockId that is not used already. */
private long getFreeBlockId() { private long getFreeBlockId() {
long id = rand.nextLong(); long id = rand.nextLong();
while (true) { while (true) {
@ -244,14 +243,15 @@ public class TestDirectoryScanner {
+ Block.METADATA_EXTENSION; + Block.METADATA_EXTENSION;
} }
/** Create a block file in a random volume*/ /** Create a block file in a random volume. */
private long createBlockFile() throws IOException { private long createBlockFile() throws IOException {
long id = getFreeBlockId(); long id = getFreeBlockId();
try (FsDatasetSpi.FsVolumeReferences volumes = fds.getFsVolumeReferences()) { try (
FsDatasetSpi.FsVolumeReferences volumes = fds.getFsVolumeReferences()) {
int numVolumes = volumes.size(); int numVolumes = volumes.size();
int index = rand.nextInt(numVolumes - 1); int index = rand.nextInt(numVolumes - 1);
File finalizedDir = ((FsVolumeImpl) volumes.get(index)) File finalizedDir =
.getFinalizedDir(bpid); ((FsVolumeImpl) volumes.get(index)).getFinalizedDir(bpid);
File file = new File(finalizedDir, getBlockFile(id)); File file = new File(finalizedDir, getBlockFile(id));
if (file.createNewFile()) { if (file.createNewFile()) {
LOG.info("Created block file " + file.getName()); LOG.info("Created block file " + file.getName());
@ -260,14 +260,14 @@ public class TestDirectoryScanner {
return id; return id;
} }
/** Create a metafile in a random volume*/ /** Create a metafile in a random volume */
private long createMetaFile() throws IOException { private long createMetaFile() throws IOException {
long id = getFreeBlockId(); long id = getFreeBlockId();
try (FsDatasetSpi.FsVolumeReferences refs = fds.getFsVolumeReferences()) { try (FsDatasetSpi.FsVolumeReferences refs = fds.getFsVolumeReferences()) {
int numVolumes = refs.size(); int numVolumes = refs.size();
int index = rand.nextInt(numVolumes - 1); int index = rand.nextInt(numVolumes - 1);
File finalizedDir = ((FsVolumeImpl) refs.get(index)) File finalizedDir =
.getFinalizedDir(bpid); ((FsVolumeImpl) refs.get(index)).getFinalizedDir(bpid);
File file = new File(finalizedDir, getMetaFile(id)); File file = new File(finalizedDir, getMetaFile(id));
if (file.createNewFile()) { if (file.createNewFile()) {
LOG.info("Created metafile " + file.getName()); LOG.info("Created metafile " + file.getName());
@ -276,7 +276,7 @@ public class TestDirectoryScanner {
return id; return id;
} }
/** Create block file and corresponding metafile in a rondom volume */ /** Create block file and corresponding metafile in a rondom volume. */
private long createBlockMetaFile() throws IOException { private long createBlockMetaFile() throws IOException {
long id = getFreeBlockId(); long id = getFreeBlockId();
@ -318,7 +318,7 @@ public class TestDirectoryScanner {
long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks) long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks)
throws IOException, InterruptedException, TimeoutException { throws IOException, InterruptedException, TimeoutException {
scan(totalBlocks, diffsize, missingMetaFile, missingBlockFile, scan(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
missingMemoryBlocks, mismatchBlocks, 0); missingMemoryBlocks, mismatchBlocks, 0);
} }
private void scan(long totalBlocks, int diffsize, long missingMetaFile, private void scan(long totalBlocks, int diffsize, long missingMetaFile,
@ -332,22 +332,22 @@ public class TestDirectoryScanner {
verifyStats(totalBlocks, diffsize, missingMetaFile, missingBlockFile, verifyStats(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
missingMemoryBlocks, mismatchBlocks, duplicateBlocks); missingMemoryBlocks, mismatchBlocks, duplicateBlocks);
} catch (AssertionError ex) { } catch (AssertionError ex) {
LOG.warn("Assertion Error", ex);
return false; return false;
} }
return true; return true;
}, 50, 2000); }, 100, 2000);
} }
private void verifyStats(long totalBlocks, int diffsize, long missingMetaFile, private void verifyStats(long totalBlocks, int diffsize, long missingMetaFile,
long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks, long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks,
long duplicateBlocks) { long duplicateBlocks) {
assertTrue(scanner.diffs.containsKey(bpid)); Collection<FsVolumeSpi.ScanInfo> diff = scanner.diffs.getScanInfo(bpid);
LinkedList<FsVolumeSpi.ScanInfo> diff = scanner.diffs.get(bpid);
assertTrue(scanner.stats.containsKey(bpid));
DirectoryScanner.Stats stats = scanner.stats.get(bpid);
assertEquals(diffsize, diff.size()); assertEquals(diffsize, diff.size());
DirectoryScanner.Stats stats = scanner.stats.get(bpid);
assertNotNull(stats);
assertEquals(totalBlocks, stats.totalBlocks); assertEquals(totalBlocks, stats.totalBlocks);
assertEquals(missingMetaFile, stats.missingMetaFile); assertEquals(missingMetaFile, stats.missingMetaFile);
assertEquals(missingBlockFile, stats.missingBlockFile); assertEquals(missingBlockFile, stats.missingBlockFile);
@ -356,20 +356,18 @@ public class TestDirectoryScanner {
assertEquals(duplicateBlocks, stats.duplicateBlocks); assertEquals(duplicateBlocks, stats.duplicateBlocks);
} }
@Test (timeout=300000) @Test(timeout = 300000)
public void testRetainBlockOnPersistentStorage() throws Exception { public void testRetainBlockOnPersistentStorage() throws Exception {
cluster = new MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF)
.Builder(CONF) .storageTypes(
.storageTypes(new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT }) new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT })
.numDataNodes(1) .numDataNodes(1).build();
.build();
try { try {
cluster.waitActive(); cluster.waitActive();
DataNode dataNode = cluster.getDataNodes().get(0);
bpid = cluster.getNamesystem().getBlockPoolId(); bpid = cluster.getNamesystem().getBlockPoolId();
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0)); fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
client = cluster.getFileSystem().getClient(); client = cluster.getFileSystem().getClient();
scanner = new DirectoryScanner(dataNode, fds, CONF); scanner = new DirectoryScanner(fds, CONF);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0)); FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
@ -397,24 +395,22 @@ public class TestDirectoryScanner {
} }
} }
@Test (timeout=300000) @Test(timeout = 300000)
public void testDeleteBlockOnTransientStorage() throws Exception { public void testDeleteBlockOnTransientStorage() throws Exception {
cluster = new MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF)
.Builder(CONF) .storageTypes(
.storageTypes(new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT }) new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT })
.numDataNodes(1) .numDataNodes(1).build();
.build();
try { try {
cluster.waitActive(); cluster.waitActive();
bpid = cluster.getNamesystem().getBlockPoolId(); bpid = cluster.getNamesystem().getBlockPoolId();
DataNode dataNode = cluster.getDataNodes().get(0);
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0)); fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
client = cluster.getFileSystem().getClient(); client = cluster.getFileSystem().getClient();
scanner = new DirectoryScanner(dataNode, fds, CONF); scanner = new DirectoryScanner(fds, CONF);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0)); FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
// Create a file file on RAM_DISK // Create a file on RAM_DISK
List<LocatedBlock> blocks = List<LocatedBlock> blocks =
createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH, true); createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH, true);
@ -440,14 +436,14 @@ public class TestDirectoryScanner {
} }
} }
@Test (timeout=600000) @Test(timeout = 600000)
public void testDirectoryScanner() throws Exception { public void testDirectoryScanner() throws Exception {
// Run the test with and without parallel scanning // Run the test with and without parallel scanning
for (int parallelism = 1; parallelism < 3; parallelism++) { for (int parallelism = 1; parallelism < 3; parallelism++) {
runTest(parallelism); runTest(parallelism);
} }
} }
public void runTest(int parallelism) throws Exception { public void runTest(int parallelism) throws Exception {
cluster = new MiniDFSCluster.Builder(CONF).build(); cluster = new MiniDFSCluster.Builder(CONF).build();
try { try {
@ -456,9 +452,9 @@ public class TestDirectoryScanner {
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0)); fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
client = cluster.getFileSystem().getClient(); client = cluster.getFileSystem().getClient();
CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
parallelism); parallelism);
DataNode dataNode = cluster.getDataNodes().get(0);
scanner = new DirectoryScanner(dataNode, fds, CONF); scanner = new DirectoryScanner(fds, CONF);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
// Add files with 100 blocks // Add files with 100 blocks
@ -492,7 +488,7 @@ public class TestDirectoryScanner {
// Test5: A metafile exists for which there is no block file and // Test5: A metafile exists for which there is no block file and
// a block in memory // a block in memory
blockId = createMetaFile(); blockId = createMetaFile();
scan(totalBlocks+1, 1, 0, 1, 1, 0); scan(totalBlocks + 1, 1, 0, 1, 1, 0);
File metafile = new File(getMetaFile(blockId)); File metafile = new File(getMetaFile(blockId));
assertTrue(!metafile.exists()); assertTrue(!metafile.exists());
scan(totalBlocks, 0, 0, 0, 0, 0); scan(totalBlocks, 0, 0, 0, 0, 0);
@ -521,7 +517,7 @@ public class TestDirectoryScanner {
scan(totalBlocks, 0, 0, 0, 0, 0); scan(totalBlocks, 0, 0, 0, 0, 0);
// Test9: create a bunch of blocks files // Test9: create a bunch of blocks files
for (int i = 0; i < 10 ; i++) { for (int i = 0; i < 10; i++) {
blockId = createBlockFile(); blockId = createBlockFile();
} }
totalBlocks += 10; totalBlocks += 10;
@ -529,14 +525,14 @@ public class TestDirectoryScanner {
scan(totalBlocks, 0, 0, 0, 0, 0); scan(totalBlocks, 0, 0, 0, 0, 0);
// Test10: create a bunch of metafiles // Test10: create a bunch of metafiles
for (int i = 0; i < 10 ; i++) { for (int i = 0; i < 10; i++) {
blockId = createMetaFile(); blockId = createMetaFile();
} }
scan(totalBlocks+10, 10, 0, 10, 10, 0); scan(totalBlocks + 10, 10, 0, 10, 10, 0);
scan(totalBlocks, 0, 0, 0, 0, 0); scan(totalBlocks, 0, 0, 0, 0, 0);
// Test11: create a bunch block files and meta files // Test11: create a bunch block files and meta files
for (int i = 0; i < 10 ; i++) { for (int i = 0; i < 10; i++) {
blockId = createBlockMetaFile(); blockId = createBlockMetaFile();
} }
totalBlocks += 10; totalBlocks += 10;
@ -544,7 +540,7 @@ public class TestDirectoryScanner {
scan(totalBlocks, 0, 0, 0, 0, 0); scan(totalBlocks, 0, 0, 0, 0, 0);
// Test12: truncate block files to test block length mismatch // Test12: truncate block files to test block length mismatch
for (int i = 0; i < 10 ; i++) { for (int i = 0; i < 10; i++) {
truncateBlockFile(); truncateBlockFile();
} }
scan(totalBlocks, 10, 0, 0, 0, 10); scan(totalBlocks, 10, 0, 0, 0, 10);
@ -557,9 +553,9 @@ public class TestDirectoryScanner {
deleteMetaFile(); deleteMetaFile();
deleteBlockFile(); deleteBlockFile();
truncateBlockFile(); truncateBlockFile();
scan(totalBlocks+3, 6, 2, 2, 3, 2); scan(totalBlocks + 3, 6, 2, 2, 3, 2);
scan(totalBlocks+1, 0, 0, 0, 0, 0); scan(totalBlocks + 1, 0, 0, 0, 0, 0);
// Test14: make sure no throttling is happening // Test14: make sure no throttling is happening
assertTrue("Throttle appears to be engaged", assertTrue("Throttle appears to be engaged",
scanner.timeWaitingMs.get() < 10L); scanner.timeWaitingMs.get() < 10L);
@ -567,10 +563,11 @@ public class TestDirectoryScanner {
scanner.timeRunningMs.get() > 0L); scanner.timeRunningMs.get() > 0L);
// Test15: validate clean shutdown of DirectoryScanner // Test15: validate clean shutdown of DirectoryScanner
////assertTrue(scanner.getRunStatus()); //assumes "real" FSDataset, not sim //// assertTrue(scanner.getRunStatus()); //assumes "real" FSDataset, not
// sim
scanner.shutdown(); scanner.shutdown();
assertFalse(scanner.getRunStatus()); assertFalse(scanner.getRunStatus());
} finally { } finally {
if (scanner != null) { if (scanner != null) {
scanner.shutdown(); scanner.shutdown();
@ -582,17 +579,17 @@ public class TestDirectoryScanner {
/** /**
* Test that the timeslice throttle limits the report compiler thread's * Test that the timeslice throttle limits the report compiler thread's
* execution time correctly. We test by scanning a large block pool and * execution time correctly. We test by scanning a large block pool and
* comparing the time spent waiting to the time spent running. * comparing the time spent waiting to the time spent running.
* *
* The block pool has to be large, or the ratio will be off. The throttle * The block pool has to be large, or the ratio will be off. The throttle
* allows the report compiler thread to finish its current cycle when * allows the report compiler thread to finish its current cycle when blocking
* blocking it, so the ratio will always be a little lower than expected. * it, so the ratio will always be a little lower than expected. The smaller
* The smaller the block pool, the further off the ratio will be. * the block pool, the further off the ratio will be.
* *
* @throws Exception thrown on unexpected failure * @throws Exception thrown on unexpected failure
*/ */
@Test (timeout=600000) @Test(timeout = 600000)
public void testThrottling() throws Exception { public void testThrottling() throws Exception {
Configuration conf = new Configuration(CONF); Configuration conf = new Configuration(CONF);
@ -611,10 +608,9 @@ public class TestDirectoryScanner {
conf.setInt( conf.setInt(
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY, DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
100); 100);
DataNode dataNode = cluster.getDataNodes().get(0);
final int maxBlocksPerFile = (int) DFSConfigKeys final int maxBlocksPerFile =
.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT; (int) DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT;
int numBlocksToCreate = blocks; int numBlocksToCreate = blocks;
while (numBlocksToCreate > 0) { while (numBlocksToCreate > 0) {
final int toCreate = Math.min(maxBlocksPerFile, numBlocksToCreate); final int toCreate = Math.min(maxBlocksPerFile, numBlocksToCreate);
@ -627,7 +623,7 @@ public class TestDirectoryScanner {
int retries = maxRetries; int retries = maxRetries;
while ((retries > 0) && ((ratio < 7f) || (ratio > 10f))) { while ((retries > 0) && ((ratio < 7f) || (ratio > 10f))) {
scanner = new DirectoryScanner(dataNode, fds, conf); scanner = new DirectoryScanner(fds, conf);
ratio = runThrottleTest(blocks); ratio = runThrottleTest(blocks);
retries -= 1; retries -= 1;
} }
@ -645,7 +641,7 @@ public class TestDirectoryScanner {
retries = maxRetries; retries = maxRetries;
while ((retries > 0) && ((ratio < 2.75f) || (ratio > 4.5f))) { while ((retries > 0) && ((ratio < 2.75f) || (ratio > 4.5f))) {
scanner = new DirectoryScanner(dataNode, fds, conf); scanner = new DirectoryScanner(fds, conf);
ratio = runThrottleTest(blocks); ratio = runThrottleTest(blocks);
retries -= 1; retries -= 1;
} }
@ -664,7 +660,7 @@ public class TestDirectoryScanner {
retries = maxRetries; retries = maxRetries;
while ((retries > 0) && ((ratio < 7f) || (ratio > 10f))) { while ((retries > 0) && ((ratio < 7f) || (ratio > 10f))) {
scanner = new DirectoryScanner(dataNode, fds, conf); scanner = new DirectoryScanner(fds, conf);
ratio = runThrottleTest(blocks); ratio = runThrottleTest(blocks);
retries -= 1; retries -= 1;
} }
@ -675,7 +671,7 @@ public class TestDirectoryScanner {
assertTrue("Throttle is too permissive", ratio >= 7f); assertTrue("Throttle is too permissive", ratio >= 7f);
// Test with no limit // Test with no limit
scanner = new DirectoryScanner(dataNode, fds, CONF); scanner = new DirectoryScanner(fds, CONF);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
scan(blocks, 0, 0, 0, 0, 0); scan(blocks, 0, 0, 0, 0, 0);
scanner.shutdown(); scanner.shutdown();
@ -686,7 +682,7 @@ public class TestDirectoryScanner {
assertTrue("Report complier threads logged no execution time", assertTrue("Report complier threads logged no execution time",
scanner.timeRunningMs.get() > 0L); scanner.timeRunningMs.get() > 0L);
// Test with a 1ms limit. This also tests whether the scanner can be // Test with a 1ms limit. This also tests whether the scanner can be
// shutdown cleanly in mid stride. // shutdown cleanly in mid stride.
conf.setInt( conf.setInt(
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY, DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
@ -698,7 +694,7 @@ public class TestDirectoryScanner {
try { try {
while ((retries > 0) && (ratio < 10)) { while ((retries > 0) && (ratio < 10)) {
scanner = new DirectoryScanner(dataNode, fds, conf); scanner = new DirectoryScanner(fds, conf);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
final AtomicLong nowMs = new AtomicLong(); final AtomicLong nowMs = new AtomicLong();
@ -728,7 +724,7 @@ public class TestDirectoryScanner {
} }
ratio = ratio =
(float)scanner.timeWaitingMs.get() / scanner.timeRunningMs.get(); (float) scanner.timeWaitingMs.get() / scanner.timeRunningMs.get();
retries -= 1; retries -= 1;
} }
} finally { } finally {
@ -737,8 +733,7 @@ public class TestDirectoryScanner {
// We just want to test that it waits a lot, but it also runs some // We just want to test that it waits a lot, but it also runs some
LOG.info("RATIO: " + ratio); LOG.info("RATIO: " + ratio);
assertTrue("Throttle is too permissive", assertTrue("Throttle is too permissive", ratio > 8);
ratio > 10);
assertTrue("Report complier threads logged no execution time", assertTrue("Report complier threads logged no execution time",
scanner.timeRunningMs.get() > 0L); scanner.timeRunningMs.get() > 0L);
@ -746,7 +741,7 @@ public class TestDirectoryScanner {
conf.setInt( conf.setInt(
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY, DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
0); 0);
scanner = new DirectoryScanner(dataNode, fds, conf); scanner = new DirectoryScanner(fds, conf);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
scan(blocks, 0, 0, 0, 0, 0); scan(blocks, 0, 0, 0, 0, 0);
scanner.shutdown(); scanner.shutdown();
@ -761,7 +756,7 @@ public class TestDirectoryScanner {
conf.setInt( conf.setInt(
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY, DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
1000); 1000);
scanner = new DirectoryScanner(dataNode, fds, conf); scanner = new DirectoryScanner(fds, conf);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
scan(blocks, 0, 0, 0, 0, 0); scan(blocks, 0, 0, 0, 0, 0);
scanner.shutdown(); scanner.shutdown();
@ -777,9 +772,8 @@ public class TestDirectoryScanner {
conf.setInt( conf.setInt(
DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY, DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY,
10); 10);
conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
1); scanner = new DirectoryScanner(fds, conf);
scanner = new DirectoryScanner(dataNode, fds, conf);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
scanner.start(); scanner.start();
@ -805,7 +799,7 @@ public class TestDirectoryScanner {
scanner.shutdown(); scanner.shutdown();
assertFalse(scanner.getRunStatus()); assertFalse(scanner.getRunStatus());
return (float)scanner.timeWaitingMs.get() / scanner.timeRunningMs.get(); return (float) scanner.timeWaitingMs.get() / scanner.timeRunningMs.get();
} }
private void verifyAddition(long blockId, long genStamp, long size) { private void verifyAddition(long blockId, long genStamp, long size) {
@ -836,7 +830,7 @@ public class TestDirectoryScanner {
assertNotNull(memBlock); assertNotNull(memBlock);
assertEquals(genStamp, memBlock.getGenerationStamp()); assertEquals(genStamp, memBlock.getGenerationStamp());
} }
private void verifyStorageType(long blockId, boolean expectTransient) { private void verifyStorageType(long blockId, boolean expectTransient) {
final ReplicaInfo memBlock; final ReplicaInfo memBlock;
memBlock = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId); memBlock = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
@ -859,7 +853,7 @@ public class TestDirectoryScanner {
public long getAvailable() throws IOException { public long getAvailable() throws IOException {
return 0; return 0;
} }
public File getFinalizedDir(String bpid) throws IOException { public File getFinalizedDir(String bpid) throws IOException {
return new File("/base/current/" + bpid + "/finalized"); return new File("/base/current/" + bpid + "/finalized");
} }
@ -898,10 +892,11 @@ public class TestDirectoryScanner {
@Override @Override
public BlockIterator loadBlockIterator(String bpid, String name) public BlockIterator loadBlockIterator(String bpid, String name)
throws IOException { throws IOException {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@SuppressWarnings("rawtypes")
@Override @Override
public FsDatasetSpi getDataset() { public FsDatasetSpi getDataset() {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
@ -923,8 +918,8 @@ public class TestDirectoryScanner {
} }
@Override @Override
public byte[] loadLastPartialChunkChecksum( public byte[] loadLastPartialChunkChecksum(File blockFile, File metaFile)
File blockFile, File metaFile) throws IOException { throws IOException {
return null; return null;
} }
@ -945,7 +940,6 @@ public class TestDirectoryScanner {
return null; return null;
} }
@Override @Override
public VolumeCheckResult check(VolumeCheckContext context) public VolumeCheckResult check(VolumeCheckContext context)
throws Exception { throws Exception {
@ -954,11 +948,11 @@ public class TestDirectoryScanner {
} }
private final static TestFsVolumeSpi TEST_VOLUME = new TestFsVolumeSpi(); private final static TestFsVolumeSpi TEST_VOLUME = new TestFsVolumeSpi();
private final static String BPID_1 = "BP-783049782-127.0.0.1-1370971773491"; private final static String BPID_1 = "BP-783049782-127.0.0.1-1370971773491";
private final static String BPID_2 = "BP-367845636-127.0.0.1-5895645674231"; private final static String BPID_2 = "BP-367845636-127.0.0.1-5895645674231";
void testScanInfoObject(long blockId, File blockFile, File metaFile) void testScanInfoObject(long blockId, File blockFile, File metaFile)
throws Exception { throws Exception {
FsVolumeSpi.ScanInfo scanInfo = FsVolumeSpi.ScanInfo scanInfo =
@ -978,7 +972,7 @@ public class TestDirectoryScanner {
} }
assertEquals(TEST_VOLUME, scanInfo.getVolume()); assertEquals(TEST_VOLUME, scanInfo.getVolume());
} }
void testScanInfoObject(long blockId) throws Exception { void testScanInfoObject(long blockId) throws Exception {
FsVolumeSpi.ScanInfo scanInfo = FsVolumeSpi.ScanInfo scanInfo =
new FsVolumeSpi.ScanInfo(blockId, null, null, null); new FsVolumeSpi.ScanInfo(blockId, null, null, null);
@ -987,7 +981,7 @@ public class TestDirectoryScanner {
assertNull(scanInfo.getMetaFile()); assertNull(scanInfo.getMetaFile());
} }
@Test(timeout=120000) @Test(timeout = 120000)
public void TestScanInfo() throws Exception { public void TestScanInfo() throws Exception {
testScanInfoObject(123, testScanInfoObject(123,
new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(), new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(),
@ -998,13 +992,10 @@ public class TestDirectoryScanner {
new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(), new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(),
"blk_123"), "blk_123"),
null); null);
testScanInfoObject(523, testScanInfoObject(523, null,
null,
new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(), new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath(),
"blk_123__1009.meta")); "blk_123__1009.meta"));
testScanInfoObject(789, testScanInfoObject(789, null, null);
null,
null);
testScanInfoObject(456); testScanInfoObject(456);
testScanInfoObject(123, testScanInfoObject(123,
new File(TEST_VOLUME.getFinalizedDir(BPID_2).getAbsolutePath(), new File(TEST_VOLUME.getFinalizedDir(BPID_2).getAbsolutePath(),
@ -1027,7 +1018,6 @@ public class TestDirectoryScanner {
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0)); fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
client = cluster.getFileSystem().getClient(); client = cluster.getFileSystem().getClient();
CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1); CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
DataNode dataNode = cluster.getDataNodes().get(0);
// Add files with 2 blocks // Add files with 2 blocks
createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 2, false); createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 2, false);
@ -1047,7 +1037,7 @@ public class TestDirectoryScanner {
FsDatasetSpi<? extends FsVolumeSpi> spyFds = Mockito.spy(fds); FsDatasetSpi<? extends FsVolumeSpi> spyFds = Mockito.spy(fds);
Mockito.doReturn(volReferences).when(spyFds).getFsVolumeReferences(); Mockito.doReturn(volReferences).when(spyFds).getFsVolumeReferences();
scanner = new DirectoryScanner(dataNode, spyFds, CONF); scanner = new DirectoryScanner(spyFds, CONF);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
scanner.reconcile(); scanner.reconcile();
} finally { } finally {
@ -1061,28 +1051,27 @@ public class TestDirectoryScanner {
@Test @Test
public void testDirectoryScannerInFederatedCluster() throws Exception { public void testDirectoryScannerInFederatedCluster() throws Exception {
//Create Federated cluster with two nameservices and one DN // Create Federated cluster with two nameservices and one DN
try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF) try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF)
.nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2)) .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))
.numDataNodes(1).build()) { .numDataNodes(1).build()) {
cluster.waitActive(); cluster.waitActive();
cluster.transitionToActive(1); cluster.transitionToActive(1);
cluster.transitionToActive(3); cluster.transitionToActive(3);
DataNode dataNode = cluster.getDataNodes().get(0);
fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0)); fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
//Create one block in first nameservice // Create one block in first nameservice
FileSystem fs = cluster.getFileSystem(1); FileSystem fs = cluster.getFileSystem(1);
int bp1Files = 1; int bp1Files = 1;
writeFile(fs, bp1Files); writeFile(fs, bp1Files);
//Create two blocks in second nameservice // Create two blocks in second nameservice
FileSystem fs2 = cluster.getFileSystem(3); FileSystem fs2 = cluster.getFileSystem(3);
int bp2Files = 2; int bp2Files = 2;
writeFile(fs2, bp2Files); writeFile(fs2, bp2Files);
//Call the Directory scanner // Call the Directory scanner
scanner = new DirectoryScanner(dataNode, fds, CONF); scanner = new DirectoryScanner(fds, CONF);
scanner.setRetainDiffs(true); scanner.setRetainDiffs(true);
scanner.reconcile(); scanner.reconcile();
//Check blocks in corresponding BP // Check blocks in corresponding BP
GenericTestUtils.waitFor(() -> { GenericTestUtils.waitFor(() -> {
try { try {

View File

@ -34,6 +34,7 @@ import java.io.Writer;
import java.net.URI; import java.net.URI;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
@ -66,6 +67,7 @@ import org.apache.hadoop.hdfs.server.datanode.DNConf;
import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataStorage; import org.apache.hadoop.hdfs.server.datanode.DataStorage;
import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner; import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner;
import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ScanInfoVolumeReport;
import org.apache.hadoop.hdfs.server.datanode.FinalizedProvidedReplica; import org.apache.hadoop.hdfs.server.datanode.FinalizedProvidedReplica;
import org.apache.hadoop.hdfs.server.datanode.ProvidedReplica; import org.apache.hadoop.hdfs.server.datanode.ProvidedReplica;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo; import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
@ -73,9 +75,9 @@ import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation; import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdfs.server.datanode.TestProvidedReplicaImpl; import org.apache.hadoop.hdfs.server.datanode.TestProvidedReplicaImpl;
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.FsDatasetSpi.FsVolumeReferences;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.BlockIterator; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.BlockIterator;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.AutoCloseableLock; import org.apache.hadoop.util.AutoCloseableLock;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
@ -183,7 +185,6 @@ public class TestProvidedImpl {
public static class TestFileRegionBlockAliasMap public static class TestFileRegionBlockAliasMap
extends BlockAliasMap<FileRegion> { extends BlockAliasMap<FileRegion> {
private Configuration conf;
private int minId; private int minId;
private int numBlocks; private int numBlocks;
private Iterator<FileRegion> suppliedIterator; private Iterator<FileRegion> suppliedIterator;
@ -592,11 +593,13 @@ public class TestProvidedImpl {
@Test @Test
public void testScannerWithProvidedVolumes() throws Exception { public void testScannerWithProvidedVolumes() throws Exception {
DirectoryScanner scanner = new DirectoryScanner(datanode, dataset, conf); DirectoryScanner scanner = new DirectoryScanner(dataset, conf);
Map<String, FsVolumeSpi.ScanInfo[]> report = scanner.getDiskReport(); Collection<ScanInfoVolumeReport> reports = scanner.getVolumeReports();
// no blocks should be reported for the Provided volume as long as // no blocks should be reported for the Provided volume as long as
// the directoryScanner is disabled. // the directoryScanner is disabled.
assertEquals(0, report.get(BLOCK_POOL_IDS[CHOSEN_BP_ID]).length); for (ScanInfoVolumeReport report : reports) {
assertEquals(0, report.getScanInfo(BLOCK_POOL_IDS[CHOSEN_BP_ID]).size());
}
} }
/** /**

View File

@ -17,6 +17,7 @@
*/ */
package org.apache.hadoop.hdfs.server.namenode; package org.apache.hadoop.hdfs.server.namenode;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.File; import java.io.File;
@ -63,14 +64,19 @@ public class TestListCorruptFileBlocks {
@Test (timeout=300000) @Test (timeout=300000)
public void testListCorruptFilesCorruptedBlock() throws Exception { public void testListCorruptFilesCorruptedBlock() throws Exception {
MiniDFSCluster cluster = null; MiniDFSCluster cluster = null;
Random random = new Random();
try { try {
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1); // datanode scans directories
conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000); // datanode sends block reports // datanode scans directories
conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
// datanode sends block reports
conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000);
// Set short retry timeouts so this test runs faster // Set short retry timeouts so this test runs faster
conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10); conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
cluster = new MiniDFSCluster.Builder(conf).build(); cluster = new MiniDFSCluster.Builder(conf).build();
FileSystem fs = cluster.getFileSystem(); FileSystem fs = cluster.getFileSystem();
@ -84,8 +90,8 @@ public class TestListCorruptFileBlocks {
final NameNode namenode = cluster.getNameNode(); final NameNode namenode = cluster.getNameNode();
Collection<FSNamesystem.CorruptFileBlockInfo> badFiles = namenode. Collection<FSNamesystem.CorruptFileBlockInfo> badFiles = namenode.
getNamesystem().listCorruptFileBlocks("/", null); getNamesystem().listCorruptFileBlocks("/", null);
assertTrue("Namenode has " + badFiles.size() assertEquals("Namenode has " + badFiles.size()
+ " corrupt files. Expecting None.", badFiles.size() == 0); + " corrupt files. Expecting None.", 0, badFiles.size());
// Now deliberately corrupt one block // Now deliberately corrupt one block
String bpid = cluster.getNamesystem().getBlockPoolId(); String bpid = cluster.getNamesystem().getBlockPoolId();
@ -101,7 +107,7 @@ public class TestListCorruptFileBlocks {
long position = channel.size() - 2; long position = channel.size() - 2;
int length = 2; int length = 2;
byte[] buffer = new byte[length]; byte[] buffer = new byte[length];
random.nextBytes(buffer); new Random(13L).nextBytes(buffer);
channel.write(ByteBuffer.wrap(buffer), position); channel.write(ByteBuffer.wrap(buffer), position);
file.close(); file.close();
LOG.info("Deliberately corrupting file " + metaFile.getName() + LOG.info("Deliberately corrupting file " + metaFile.getName() +
@ -134,7 +140,6 @@ public class TestListCorruptFileBlocks {
@Test (timeout=300000) @Test (timeout=300000)
public void testListCorruptFileBlocksInSafeMode() throws Exception { public void testListCorruptFileBlocksInSafeMode() throws Exception {
MiniDFSCluster cluster = null; MiniDFSCluster cluster = null;
Random random = new Random();
try { try {
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
@ -164,8 +169,8 @@ public class TestListCorruptFileBlocks {
// fetch bad file list from namenode. There should be none. // fetch bad file list from namenode. There should be none.
Collection<FSNamesystem.CorruptFileBlockInfo> badFiles = Collection<FSNamesystem.CorruptFileBlockInfo> badFiles =
cluster.getNameNode().getNamesystem().listCorruptFileBlocks("/", null); cluster.getNameNode().getNamesystem().listCorruptFileBlocks("/", null);
assertTrue("Namenode has " + badFiles.size() assertEquals("Namenode has " + badFiles.size()
+ " corrupt files. Expecting None.", badFiles.size() == 0); + " corrupt files. Expecting None.", 0, badFiles.size());
// Now deliberately corrupt one block // Now deliberately corrupt one block
File storageDir = cluster.getInstanceStorageDir(0, 0); File storageDir = cluster.getInstanceStorageDir(0, 0);
@ -181,7 +186,7 @@ public class TestListCorruptFileBlocks {
long position = channel.size() - 2; long position = channel.size() - 2;
int length = 2; int length = 2;
byte[] buffer = new byte[length]; byte[] buffer = new byte[length];
random.nextBytes(buffer); new Random(13L).nextBytes(buffer);
channel.write(ByteBuffer.wrap(buffer), position); channel.write(ByteBuffer.wrap(buffer), position);
file.close(); file.close();
LOG.info("Deliberately corrupting file " + metaFile.getName() + LOG.info("Deliberately corrupting file " + metaFile.getName() +
@ -318,9 +323,9 @@ public class TestListCorruptFileBlocks {
} }
// Validate we get all the corrupt files // Validate we get all the corrupt files
LOG.info("Namenode has bad files. " + numCorrupt); LOG.info("Namenode has bad files. " + numCorrupt);
assertTrue(numCorrupt == 3); assertEquals(3, numCorrupt);
// test the paging here
// test the paging here
FSNamesystem.CorruptFileBlockInfo[] cfb = corruptFileBlocks FSNamesystem.CorruptFileBlockInfo[] cfb = corruptFileBlocks
.toArray(new FSNamesystem.CorruptFileBlockInfo[0]); .toArray(new FSNamesystem.CorruptFileBlockInfo[0]);
// now get the 2nd and 3rd file that is corrupt // now get the 2nd and 3rd file that is corrupt
@ -331,7 +336,7 @@ public class TestListCorruptFileBlocks {
FSNamesystem.CorruptFileBlockInfo[] ncfb = nextCorruptFileBlocks FSNamesystem.CorruptFileBlockInfo[] ncfb = nextCorruptFileBlocks
.toArray(new FSNamesystem.CorruptFileBlockInfo[0]); .toArray(new FSNamesystem.CorruptFileBlockInfo[0]);
numCorrupt = nextCorruptFileBlocks.size(); numCorrupt = nextCorruptFileBlocks.size();
assertTrue(numCorrupt == 2); assertEquals(2, numCorrupt);
assertTrue(ncfb[0].block.getBlockName() assertTrue(ncfb[0].block.getBlockName()
.equalsIgnoreCase(cfb[1].block.getBlockName())); .equalsIgnoreCase(cfb[1].block.getBlockName()));
@ -339,14 +344,14 @@ public class TestListCorruptFileBlocks {
namenode.getNamesystem() namenode.getNamesystem()
.listCorruptFileBlocks("/corruptData", cookie); .listCorruptFileBlocks("/corruptData", cookie);
numCorrupt = corruptFileBlocks.size(); numCorrupt = corruptFileBlocks.size();
assertTrue(numCorrupt == 0); assertEquals(0, numCorrupt);
// Do a listing on a dir which doesn't have any corrupt blocks and // Do a listing on a dir which doesn't have any corrupt blocks and
// validate // validate
util.createFiles(fs, "/goodData"); util.createFiles(fs, "/goodData");
corruptFileBlocks = corruptFileBlocks =
namenode.getNamesystem().listCorruptFileBlocks("/goodData", null); namenode.getNamesystem().listCorruptFileBlocks("/goodData", null);
numCorrupt = corruptFileBlocks.size(); numCorrupt = corruptFileBlocks.size();
assertTrue(numCorrupt == 0); assertEquals(0, numCorrupt);
util.cleanup(fs, "/corruptData"); util.cleanup(fs, "/corruptData");
util.cleanup(fs, "/goodData"); util.cleanup(fs, "/goodData");
} finally { } finally {
@ -390,7 +395,7 @@ public class TestListCorruptFileBlocks {
RemoteIterator<Path> corruptFileBlocks = RemoteIterator<Path> corruptFileBlocks =
dfs.listCorruptFileBlocks(new Path("/corruptData")); dfs.listCorruptFileBlocks(new Path("/corruptData"));
int numCorrupt = countPaths(corruptFileBlocks); int numCorrupt = countPaths(corruptFileBlocks);
assertTrue(numCorrupt == 0); assertEquals(0, numCorrupt);
// delete the blocks // delete the blocks
String bpid = cluster.getNamesystem().getBlockPoolId(); String bpid = cluster.getNamesystem().getBlockPoolId();
// For loop through number of datadirectories per datanode (2) // For loop through number of datadirectories per datanode (2)
@ -426,7 +431,7 @@ public class TestListCorruptFileBlocks {
} }
// Validate we get all the corrupt files // Validate we get all the corrupt files
LOG.info("Namenode has bad files. " + numCorrupt); LOG.info("Namenode has bad files. " + numCorrupt);
assertTrue(numCorrupt == 3); assertEquals(3, numCorrupt);
util.cleanup(fs, "/corruptData"); util.cleanup(fs, "/corruptData");
util.cleanup(fs, "/goodData"); util.cleanup(fs, "/goodData");
@ -465,8 +470,9 @@ public class TestListCorruptFileBlocks {
final NameNode namenode = cluster.getNameNode(); final NameNode namenode = cluster.getNameNode();
Collection<FSNamesystem.CorruptFileBlockInfo> badFiles = namenode. Collection<FSNamesystem.CorruptFileBlockInfo> badFiles = namenode.
getNamesystem().listCorruptFileBlocks("/srcdat2", null); getNamesystem().listCorruptFileBlocks("/srcdat2", null);
assertTrue("Namenode has " + badFiles.size() + " corrupt files. Expecting none.", assertEquals(
badFiles.size() == 0); "Namenode has " + badFiles.size() + " corrupt files. Expecting none.",
0, badFiles.size());
// Now deliberately blocks from all files // Now deliberately blocks from all files
final String bpid = cluster.getNamesystem().getBlockPoolId(); final String bpid = cluster.getNamesystem().getBlockPoolId();
@ -555,7 +561,7 @@ public class TestListCorruptFileBlocks {
RemoteIterator<Path> corruptFileBlocks = dfs RemoteIterator<Path> corruptFileBlocks = dfs
.listCorruptFileBlocks(new Path("corruptData")); .listCorruptFileBlocks(new Path("corruptData"));
int numCorrupt = countPaths(corruptFileBlocks); int numCorrupt = countPaths(corruptFileBlocks);
assertTrue(numCorrupt == 0); assertEquals(0, numCorrupt);
// delete the blocks // delete the blocks
String bpid = cluster.getNamesystem().getBlockPoolId(); String bpid = cluster.getNamesystem().getBlockPoolId();
@ -589,7 +595,7 @@ public class TestListCorruptFileBlocks {
} }
// Validate we get all the corrupt files // Validate we get all the corrupt files
LOG.info("Namenode has bad files. " + numCorrupt); LOG.info("Namenode has bad files. " + numCorrupt);
assertTrue("Failed to get corrupt files!", numCorrupt == 3); assertEquals("Failed to get corrupt files!", 3, numCorrupt);
util.cleanup(fs, "corruptData"); util.cleanup(fs, "corruptData");
} finally { } finally {