|
@@ -44,6 +44,7 @@ import java.util.concurrent.ForkJoinTask;
|
|
import java.util.concurrent.RecursiveAction;
|
|
import java.util.concurrent.RecursiveAction;
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.hdfs.server.datanode.FSCachingGetSpaceUsed;
|
|
import org.apache.hadoop.hdfs.server.datanode.FSCachingGetSpaceUsed;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
@@ -103,7 +104,8 @@ class BlockPoolSlice {
|
|
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;
|
|
|
|
+ private final File replicaCacheDir;
|
|
private AtomicLong numOfBlocks = new AtomicLong();
|
|
private AtomicLong numOfBlocks = new AtomicLong();
|
|
private final long cachedDfsUsedCheckTime;
|
|
private final long cachedDfsUsedCheckTime;
|
|
private final Timer timer;
|
|
private final Timer timer;
|
|
@@ -180,6 +182,24 @@ class BlockPoolSlice {
|
|
fileIoProvider.mkdirs(volume, rbwDir);
|
|
fileIoProvider.mkdirs(volume, rbwDir);
|
|
fileIoProvider.mkdirs(volume, tmpDir);
|
|
fileIoProvider.mkdirs(volume, tmpDir);
|
|
|
|
|
|
|
|
+ String cacheDirRoot = conf.get(
|
|
|
|
+ DFSConfigKeys.DFS_DATANODE_REPLICA_CACHE_ROOT_DIR_KEY);
|
|
|
|
+ if (cacheDirRoot != null && !cacheDirRoot.isEmpty()) {
|
|
|
|
+ this.replicaCacheDir = new File(cacheDirRoot,
|
|
|
|
+ currentDir.getCanonicalPath());
|
|
|
|
+ if (!this.replicaCacheDir.exists()) {
|
|
|
|
+ if (!this.replicaCacheDir.mkdirs()) {
|
|
|
|
+ throw new IOException("Failed to mkdirs " + this.replicaCacheDir);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ this.replicaCacheDir = currentDir;
|
|
|
|
+ }
|
|
|
|
+ this.replicaCacheExpiry = conf.getTimeDuration(
|
|
|
|
+ DFSConfigKeys.DFS_DATANODE_REPLICA_CACHE_EXPIRY_TIME_KEY,
|
|
|
|
+ DFSConfigKeys.DFS_DATANODE_REPLICA_CACHE_EXPIRY_TIME_DEFAULT,
|
|
|
|
+ TimeUnit.MILLISECONDS);
|
|
|
|
+
|
|
// Use cached value initially if available. Or the following call will
|
|
// Use cached value initially if available. Or the following call will
|
|
// block until the initial du command completes.
|
|
// block until the initial du command completes.
|
|
this.dfsUsage = new FSCachingGetSpaceUsed.Builder().setBpid(bpid)
|
|
this.dfsUsage = new FSCachingGetSpaceUsed.Builder().setBpid(bpid)
|
|
@@ -876,7 +896,7 @@ class BlockPoolSlice {
|
|
private boolean readReplicasFromCache(ReplicaMap volumeMap,
|
|
private boolean readReplicasFromCache(ReplicaMap volumeMap,
|
|
final RamDiskReplicaTracker lazyWriteReplicaMap) {
|
|
final RamDiskReplicaTracker lazyWriteReplicaMap) {
|
|
ReplicaMap tmpReplicaMap = new ReplicaMap(new ReentrantReadWriteLock());
|
|
ReplicaMap tmpReplicaMap = new ReplicaMap(new ReentrantReadWriteLock());
|
|
- File replicaFile = new File(currentDir, REPLICA_CACHE_FILE);
|
|
|
|
|
|
+ File replicaFile = new File(replicaCacheDir, REPLICA_CACHE_FILE);
|
|
// Check whether the file exists or not.
|
|
// Check whether the file exists or not.
|
|
if (!replicaFile.exists()) {
|
|
if (!replicaFile.exists()) {
|
|
LOG.info("Replica Cache file: "+ replicaFile.getPath() +
|
|
LOG.info("Replica Cache file: "+ replicaFile.getPath() +
|
|
@@ -954,8 +974,8 @@ class BlockPoolSlice {
|
|
blocksListToPersist.getNumberOfBlocks()== 0) {
|
|
blocksListToPersist.getNumberOfBlocks()== 0) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
- final File tmpFile = new File(currentDir, REPLICA_CACHE_FILE + ".tmp");
|
|
|
|
- final File replicaCacheFile = new File(currentDir, REPLICA_CACHE_FILE);
|
|
|
|
|
|
+ final File tmpFile = new File(replicaCacheDir, REPLICA_CACHE_FILE + ".tmp");
|
|
|
|
+ final File replicaCacheFile = new File(replicaCacheDir, REPLICA_CACHE_FILE);
|
|
if (!fileIoProvider.deleteWithExistsCheck(volume, tmpFile) ||
|
|
if (!fileIoProvider.deleteWithExistsCheck(volume, tmpFile) ||
|
|
!fileIoProvider.deleteWithExistsCheck(volume, replicaCacheFile)) {
|
|
!fileIoProvider.deleteWithExistsCheck(volume, replicaCacheFile)) {
|
|
return;
|
|
return;
|