|
@@ -55,7 +55,7 @@ import org.apache.hadoop.util.DataChecksum;
|
|
import org.apache.hadoop.util.DiskChecker;
|
|
import org.apache.hadoop.util.DiskChecker;
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
-import org.apache.hadoop.util.Time;
|
|
|
|
|
|
+import org.apache.hadoop.util.Timer;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.io.Files;
|
|
import com.google.common.io.Files;
|
|
@@ -79,12 +79,15 @@ class BlockPoolSlice {
|
|
private final File rbwDir; // directory store RBW replica
|
|
private final File rbwDir; // directory store RBW replica
|
|
private final File tmpDir; // directory store Temporary replica
|
|
private final File tmpDir; // directory store Temporary replica
|
|
private final int ioFileBufferSize;
|
|
private final int ioFileBufferSize;
|
|
- private static final String DU_CACHE_FILE = "dfsUsed";
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ public static final String DU_CACHE_FILE = "dfsUsed";
|
|
private volatile boolean dfsUsedSaved = false;
|
|
private volatile boolean dfsUsedSaved = false;
|
|
private static final int SHUTDOWN_HOOK_PRIORITY = 30;
|
|
private static final int SHUTDOWN_HOOK_PRIORITY = 30;
|
|
private final boolean deleteDuplicateReplicas;
|
|
private final boolean deleteDuplicateReplicas;
|
|
private static final String REPLICA_CACHE_FILE = "replicas";
|
|
private static final String REPLICA_CACHE_FILE = "replicas";
|
|
private final long replicaCacheExpiry = 5*60*1000;
|
|
private final long replicaCacheExpiry = 5*60*1000;
|
|
|
|
+ private final long cachedDfsUsedCheckTime;
|
|
|
|
+ private final Timer timer;
|
|
|
|
|
|
// TODO:FEDERATION scalability issue - a thread per DU is needed
|
|
// TODO:FEDERATION scalability issue - a thread per DU is needed
|
|
private final DU dfsUsage;
|
|
private final DU dfsUsage;
|
|
@@ -95,10 +98,11 @@ class BlockPoolSlice {
|
|
* @param volume {@link FsVolumeImpl} to which this BlockPool belongs to
|
|
* @param volume {@link FsVolumeImpl} to which this BlockPool belongs to
|
|
* @param bpDir directory corresponding to the BlockPool
|
|
* @param bpDir directory corresponding to the BlockPool
|
|
* @param conf configuration
|
|
* @param conf configuration
|
|
|
|
+ * @param timer include methods for getting time
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
BlockPoolSlice(String bpid, FsVolumeImpl volume, File bpDir,
|
|
BlockPoolSlice(String bpid, FsVolumeImpl volume, File bpDir,
|
|
- Configuration conf) throws IOException {
|
|
|
|
|
|
+ Configuration conf, Timer timer) throws IOException {
|
|
this.bpid = bpid;
|
|
this.bpid = bpid;
|
|
this.volume = volume;
|
|
this.volume = volume;
|
|
this.currentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
|
|
this.currentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
|
|
@@ -117,6 +121,12 @@ class BlockPoolSlice {
|
|
DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION,
|
|
DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION,
|
|
DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION_DEFAULT);
|
|
DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION_DEFAULT);
|
|
|
|
|
|
|
|
+ this.cachedDfsUsedCheckTime =
|
|
|
|
+ conf.getLong(
|
|
|
|
+ DFSConfigKeys.DFS_DN_CACHED_DFSUSED_CHECK_INTERVAL_MS,
|
|
|
|
+ DFSConfigKeys.DFS_DN_CACHED_DFSUSED_CHECK_INTERVAL_DEFAULT_MS);
|
|
|
|
+ this.timer = timer;
|
|
|
|
+
|
|
// Files that were being written when the datanode was last shutdown
|
|
// Files that were being written when the datanode was last shutdown
|
|
// are now moved back to the data directory. It is possible that
|
|
// are now moved back to the data directory. It is possible that
|
|
// in the future, we might want to do some sort of datanode-local
|
|
// in the future, we might want to do some sort of datanode-local
|
|
@@ -187,11 +197,13 @@ class BlockPoolSlice {
|
|
dfsUsage.incDfsUsed(value);
|
|
dfsUsage.incDfsUsed(value);
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Read in the cached DU value and return it if it is less than 600 seconds
|
|
|
|
- * old (DU update interval). Slight imprecision of dfsUsed is not critical
|
|
|
|
- * and skipping DU can significantly shorten the startup time.
|
|
|
|
- * If the cached value is not available or too old, -1 is returned.
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Read in the cached DU value and return it if it is less than
|
|
|
|
+ * cachedDfsUsedCheckTime which is set by
|
|
|
|
+ * dfs.datanode.cached-dfsused.check.interval.ms parameter. Slight imprecision
|
|
|
|
+ * of dfsUsed is not critical and skipping DU can significantly shorten the
|
|
|
|
+ * startup time. If the cached value is not available or too old, -1 is
|
|
|
|
+ * returned.
|
|
*/
|
|
*/
|
|
long loadDfsUsed() {
|
|
long loadDfsUsed() {
|
|
long cachedDfsUsed;
|
|
long cachedDfsUsed;
|
|
@@ -219,7 +231,7 @@ class BlockPoolSlice {
|
|
}
|
|
}
|
|
|
|
|
|
// Return the cached value if mtime is okay.
|
|
// Return the cached value if mtime is okay.
|
|
- if (mtime > 0 && (Time.now() - mtime < 600000L)) {
|
|
|
|
|
|
+ if (mtime > 0 && (timer.now() - mtime < cachedDfsUsedCheckTime)) {
|
|
FsDatasetImpl.LOG.info("Cached dfsUsed found for " + currentDir + ": " +
|
|
FsDatasetImpl.LOG.info("Cached dfsUsed found for " + currentDir + ": " +
|
|
cachedDfsUsed);
|
|
cachedDfsUsed);
|
|
return cachedDfsUsed;
|
|
return cachedDfsUsed;
|
|
@@ -245,7 +257,7 @@ class BlockPoolSlice {
|
|
try (Writer out = new OutputStreamWriter(
|
|
try (Writer out = new OutputStreamWriter(
|
|
new FileOutputStream(outFile), "UTF-8")) {
|
|
new FileOutputStream(outFile), "UTF-8")) {
|
|
// mtime is written last, so that truncated writes won't be valid.
|
|
// mtime is written last, so that truncated writes won't be valid.
|
|
- out.write(Long.toString(used) + " " + Long.toString(Time.now()));
|
|
|
|
|
|
+ out.write(Long.toString(used) + " " + Long.toString(timer.now()));
|
|
out.flush();
|
|
out.flush();
|
|
}
|
|
}
|
|
} catch (IOException ioe) {
|
|
} catch (IOException ioe) {
|
|
@@ -434,7 +446,7 @@ class BlockPoolSlice {
|
|
try {
|
|
try {
|
|
sc = new Scanner(restartMeta, "UTF-8");
|
|
sc = new Scanner(restartMeta, "UTF-8");
|
|
// The restart meta file exists
|
|
// The restart meta file exists
|
|
- if (sc.hasNextLong() && (sc.nextLong() > Time.now())) {
|
|
|
|
|
|
+ if (sc.hasNextLong() && (sc.nextLong() > timer.now())) {
|
|
// It didn't expire. Load the replica as a RBW.
|
|
// It didn't expire. Load the replica as a RBW.
|
|
// We don't know the expected block length, so just use 0
|
|
// We don't know the expected block length, so just use 0
|
|
// and don't reserve any more space for writes.
|
|
// and don't reserve any more space for writes.
|