|
@@ -36,8 +36,9 @@ import org.apache.commons.io.FileUtils;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.fs.DU;
|
|
|
+import org.apache.hadoop.fs.CachingGetSpaceUsed;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
+import org.apache.hadoop.fs.GetSpaceUsed;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
@@ -62,10 +63,10 @@ import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.io.Files;
|
|
|
|
|
|
/**
|
|
|
- * A block pool slice represents a portion of a block pool stored on a volume.
|
|
|
- * Taken together, all BlockPoolSlices sharing a block pool ID across a
|
|
|
+ * A block pool slice represents a portion of a block pool stored on a volume.
|
|
|
+ * Taken together, all BlockPoolSlices sharing a block pool ID across a
|
|
|
* cluster represent a single block pool.
|
|
|
- *
|
|
|
+ *
|
|
|
* This class is synchronized by {@link FsVolumeImpl}.
|
|
|
*/
|
|
|
class BlockPoolSlice {
|
|
@@ -92,10 +93,10 @@ class BlockPoolSlice {
|
|
|
private final Timer timer;
|
|
|
|
|
|
// TODO:FEDERATION scalability issue - a thread per DU is needed
|
|
|
- private final DU dfsUsage;
|
|
|
+ private final GetSpaceUsed dfsUsage;
|
|
|
|
|
|
/**
|
|
|
- * Create a blook pool slice
|
|
|
+ * Create a blook pool slice
|
|
|
* @param bpid Block pool Id
|
|
|
* @param volume {@link FsVolumeImpl} to which this BlockPool belongs to
|
|
|
* @param bpDir directory corresponding to the BlockPool
|
|
@@ -107,7 +108,7 @@ class BlockPoolSlice {
|
|
|
Configuration conf, Timer timer) throws IOException {
|
|
|
this.bpid = bpid;
|
|
|
this.volume = volume;
|
|
|
- this.currentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
|
|
|
+ this.currentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
|
|
|
this.finalizedDir = new File(
|
|
|
currentDir, DataStorage.STORAGE_DIR_FINALIZED);
|
|
|
this.lazypersistDir = new File(currentDir, DataStorage.STORAGE_DIR_LAZY_PERSIST);
|
|
@@ -157,8 +158,10 @@ class BlockPoolSlice {
|
|
|
}
|
|
|
// Use cached value initially if available. Or the following call will
|
|
|
// block until the initial du command completes.
|
|
|
- this.dfsUsage = new DU(bpDir, conf, loadDfsUsed());
|
|
|
- this.dfsUsage.start();
|
|
|
+ this.dfsUsage = new CachingGetSpaceUsed.Builder().setPath(bpDir)
|
|
|
+ .setConf(conf)
|
|
|
+ .setInitialUsed(loadDfsUsed())
|
|
|
+ .build();
|
|
|
|
|
|
// Make the dfs usage to be saved during shutdown.
|
|
|
ShutdownHookManager.get().addShutdownHook(
|
|
@@ -179,7 +182,7 @@ class BlockPoolSlice {
|
|
|
File getFinalizedDir() {
|
|
|
return finalizedDir;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
File getLazypersistDir() {
|
|
|
return lazypersistDir;
|
|
|
}
|
|
@@ -194,17 +197,21 @@ class BlockPoolSlice {
|
|
|
|
|
|
/** Run DU on local drives. It must be synchronized from caller. */
|
|
|
void decDfsUsed(long value) {
|
|
|
- dfsUsage.decDfsUsed(value);
|
|
|
+ if (dfsUsage instanceof CachingGetSpaceUsed) {
|
|
|
+ ((CachingGetSpaceUsed)dfsUsage).incDfsUsed(-value);
|
|
|
+ }
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
long getDfsUsed() throws IOException {
|
|
|
return dfsUsage.getUsed();
|
|
|
}
|
|
|
|
|
|
void incDfsUsed(long value) {
|
|
|
- dfsUsage.incDfsUsed(value);
|
|
|
+ if (dfsUsage instanceof CachingGetSpaceUsed) {
|
|
|
+ ((CachingGetSpaceUsed)dfsUsage).incDfsUsed(value);
|
|
|
+ }
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Read in the cached DU value and return it if it is less than
|
|
|
* cachedDfsUsedCheckTime which is set by
|
|
@@ -310,7 +317,10 @@ class BlockPoolSlice {
|
|
|
}
|
|
|
File blockFile = FsDatasetImpl.moveBlockFiles(b, f, blockDir);
|
|
|
File metaFile = FsDatasetUtil.getMetaFile(blockFile, b.getGenerationStamp());
|
|
|
- dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
|
|
|
+ if (dfsUsage instanceof CachingGetSpaceUsed) {
|
|
|
+ ((CachingGetSpaceUsed) dfsUsage).incDfsUsed(
|
|
|
+ b.getNumBytes() + metaFile.length());
|
|
|
+ }
|
|
|
return blockFile;
|
|
|
}
|
|
|
|
|
@@ -337,7 +347,7 @@ class BlockPoolSlice {
|
|
|
}
|
|
|
|
|
|
|
|
|
-
|
|
|
+
|
|
|
void getVolumeMap(ReplicaMap volumeMap,
|
|
|
final RamDiskReplicaTracker lazyWriteReplicaMap)
|
|
|
throws IOException {
|
|
@@ -348,7 +358,7 @@ class BlockPoolSlice {
|
|
|
FsDatasetImpl.LOG.info(
|
|
|
"Recovered " + numRecovered + " replicas from " + lazypersistDir);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
boolean success = readReplicasFromCache(volumeMap, lazyWriteReplicaMap);
|
|
|
if (!success) {
|
|
|
// add finalized replicas
|
|
@@ -442,7 +452,7 @@ class BlockPoolSlice {
|
|
|
FileUtil.fullyDelete(source);
|
|
|
return numRecovered;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private void addReplicaToReplicasMap(Block block, ReplicaMap volumeMap,
|
|
|
final RamDiskReplicaTracker lazyWriteReplicaMap,boolean isFinalized)
|
|
|
throws IOException {
|
|
@@ -450,7 +460,7 @@ class BlockPoolSlice {
|
|
|
long blockId = block.getBlockId();
|
|
|
long genStamp = block.getGenerationStamp();
|
|
|
if (isFinalized) {
|
|
|
- newReplica = new FinalizedReplica(blockId,
|
|
|
+ newReplica = new FinalizedReplica(blockId,
|
|
|
block.getNumBytes(), genStamp, volume, DatanodeUtil
|
|
|
.idToBlockDir(finalizedDir, blockId));
|
|
|
} else {
|
|
@@ -467,7 +477,7 @@ class BlockPoolSlice {
|
|
|
// We don't know the expected block length, so just use 0
|
|
|
// and don't reserve any more space for writes.
|
|
|
newReplica = new ReplicaBeingWritten(blockId,
|
|
|
- validateIntegrityAndSetLength(file, genStamp),
|
|
|
+ validateIntegrityAndSetLength(file, genStamp),
|
|
|
genStamp, volume, file.getParentFile(), null, 0);
|
|
|
loadRwr = false;
|
|
|
}
|
|
@@ -513,7 +523,7 @@ class BlockPoolSlice {
|
|
|
incrNumBlocks();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
|
|
|
/**
|
|
|
* Add replicas under the given directory to the volume map
|
|
@@ -543,12 +553,12 @@ class BlockPoolSlice {
|
|
|
}
|
|
|
if (!Block.isBlockFilename(file))
|
|
|
continue;
|
|
|
-
|
|
|
+
|
|
|
long genStamp = FsDatasetUtil.getGenerationStampFromFile(
|
|
|
files, file);
|
|
|
long blockId = Block.filename2id(file.getName());
|
|
|
- Block block = new Block(blockId, file.length(), genStamp);
|
|
|
- addReplicaToReplicasMap(block, volumeMap, lazyWriteReplicaMap,
|
|
|
+ Block block = new Block(blockId, file.length(), genStamp);
|
|
|
+ addReplicaToReplicasMap(block, volumeMap, lazyWriteReplicaMap,
|
|
|
isFinalized);
|
|
|
}
|
|
|
}
|
|
@@ -642,11 +652,11 @@ class BlockPoolSlice {
|
|
|
|
|
|
/**
|
|
|
* Find out the number of bytes in the block that match its crc.
|
|
|
- *
|
|
|
- * This algorithm assumes that data corruption caused by unexpected
|
|
|
+ *
|
|
|
+ * This algorithm assumes that data corruption caused by unexpected
|
|
|
* datanode shutdown occurs only in the last crc chunk. So it checks
|
|
|
* only the last chunk.
|
|
|
- *
|
|
|
+ *
|
|
|
* @param blockFile the block file
|
|
|
* @param genStamp generation stamp of the block
|
|
|
* @return the number of valid bytes
|
|
@@ -673,7 +683,7 @@ class BlockPoolSlice {
|
|
|
int bytesPerChecksum = checksum.getBytesPerChecksum();
|
|
|
int checksumSize = checksum.getChecksumSize();
|
|
|
long numChunks = Math.min(
|
|
|
- (blockFileLen + bytesPerChecksum - 1)/bytesPerChecksum,
|
|
|
+ (blockFileLen + bytesPerChecksum - 1)/bytesPerChecksum,
|
|
|
(metaFileLen - crcHeaderLen)/checksumSize);
|
|
|
if (numChunks == 0) {
|
|
|
return 0;
|
|
@@ -716,17 +726,20 @@ class BlockPoolSlice {
|
|
|
IOUtils.closeStream(blockIn);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
return currentDir.getAbsolutePath();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
void shutdown(BlockListAsLongs blocksListToPersist) {
|
|
|
saveReplicas(blocksListToPersist);
|
|
|
saveDfsUsed();
|
|
|
dfsUsedSaved = true;
|
|
|
- dfsUsage.shutdown();
|
|
|
+
|
|
|
+ if (dfsUsage instanceof CachingGetSpaceUsed) {
|
|
|
+ IOUtils.cleanup(LOG, ((CachingGetSpaceUsed) dfsUsage));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private boolean readReplicasFromCache(ReplicaMap volumeMap,
|
|
@@ -735,17 +748,17 @@ class BlockPoolSlice {
|
|
|
File replicaFile = new File(currentDir, REPLICA_CACHE_FILE);
|
|
|
// Check whether the file exists or not.
|
|
|
if (!replicaFile.exists()) {
|
|
|
- LOG.info("Replica Cache file: "+ replicaFile.getPath() +
|
|
|
+ LOG.info("Replica Cache file: "+ replicaFile.getPath() +
|
|
|
" doesn't exist ");
|
|
|
return false;
|
|
|
}
|
|
|
long fileLastModifiedTime = replicaFile.lastModified();
|
|
|
if (System.currentTimeMillis() > fileLastModifiedTime + replicaCacheExpiry) {
|
|
|
- LOG.info("Replica Cache file: " + replicaFile.getPath() +
|
|
|
+ LOG.info("Replica Cache file: " + replicaFile.getPath() +
|
|
|
" has gone stale");
|
|
|
// Just to make findbugs happy
|
|
|
if (!replicaFile.delete()) {
|
|
|
- LOG.info("Replica Cache file: " + replicaFile.getPath() +
|
|
|
+ LOG.info("Replica Cache file: " + replicaFile.getPath() +
|
|
|
" cannot be deleted");
|
|
|
}
|
|
|
return false;
|
|
@@ -782,7 +795,7 @@ class BlockPoolSlice {
|
|
|
iter.remove();
|
|
|
volumeMap.add(bpid, info);
|
|
|
}
|
|
|
- LOG.info("Successfully read replica from cache file : "
|
|
|
+ LOG.info("Successfully read replica from cache file : "
|
|
|
+ replicaFile.getPath());
|
|
|
return true;
|
|
|
} catch (Exception e) {
|
|
@@ -800,10 +813,10 @@ class BlockPoolSlice {
|
|
|
// close the inputStream
|
|
|
IOUtils.closeStream(inputStream);
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
+ }
|
|
|
+
|
|
|
private void saveReplicas(BlockListAsLongs blocksListToPersist) {
|
|
|
- if (blocksListToPersist == null ||
|
|
|
+ if (blocksListToPersist == null ||
|
|
|
blocksListToPersist.getNumberOfBlocks()== 0) {
|
|
|
return;
|
|
|
}
|
|
@@ -819,7 +832,7 @@ class BlockPoolSlice {
|
|
|
replicaCacheFile.getPath());
|
|
|
return;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
FileOutputStream out = null;
|
|
|
try {
|
|
|
out = new FileOutputStream(tmpFile);
|
|
@@ -833,7 +846,7 @@ class BlockPoolSlice {
|
|
|
// and continue.
|
|
|
LOG.warn("Failed to write replicas to cache ", e);
|
|
|
if (replicaCacheFile.exists() && !replicaCacheFile.delete()) {
|
|
|
- LOG.warn("Failed to delete replicas file: " +
|
|
|
+ LOG.warn("Failed to delete replicas file: " +
|
|
|
replicaCacheFile.getPath());
|
|
|
}
|
|
|
} finally {
|