|
@@ -219,6 +219,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
final ReplicaMap volumeMap;
|
|
|
final RamDiskReplicaTracker ramDiskReplicaTracker;
|
|
|
+ final RamDiskAsyncLazyPersistService asyncLazyPersistService;
|
|
|
|
|
|
private static final int MAX_BLOCK_EVICTIONS_PER_ITERATION = 3;
|
|
|
|
|
@@ -272,10 +273,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
VolumeChoosingPolicy.class), conf);
|
|
|
volumes = new FsVolumeList(volsFailed, blockChooserImpl);
|
|
|
asyncDiskService = new FsDatasetAsyncDiskService(datanode);
|
|
|
+ asyncLazyPersistService = new RamDiskAsyncLazyPersistService(datanode);
|
|
|
|
|
|
for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
|
|
|
addVolume(dataLocations, storage.getStorageDir(idx));
|
|
|
}
|
|
|
+ setupAsyncLazyPersistThreads();
|
|
|
|
|
|
cacheManager = new FsDatasetCache(this);
|
|
|
|
|
@@ -408,6 +411,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ setupAsyncLazyPersistThreads();
|
|
|
+
|
|
|
for (int i = 0; i < volumes.size(); i++) {
|
|
|
if (successFlags[i]) {
|
|
|
succeedVolumes.add(volumes.get(i));
|
|
@@ -461,6 +466,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
storageMap.remove(sd.getStorageUuid());
|
|
|
}
|
|
|
}
|
|
|
+ setupAsyncLazyPersistThreads();
|
|
|
}
|
|
|
|
|
|
private StorageType getStorageTypeFromLocations(
|
|
@@ -1505,10 +1511,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
RamDiskReplica replicaInfo =
|
|
|
ramDiskReplicaTracker.getReplica(bpid, invalidBlks[i].getBlockId());
|
|
|
if (replicaInfo != null) {
|
|
|
- if (replicaInfo.getIsPersisted() == false) {
|
|
|
+ if (!replicaInfo.getIsPersisted()) {
|
|
|
datanode.getMetrics().incrRamDiskBlocksDeletedBeforeLazyPersisted();
|
|
|
}
|
|
|
- discardRamDiskReplica(replicaInfo, true);
|
|
|
+ ramDiskReplicaTracker.discardReplica(replicaInfo.getBlockPoolId(),
|
|
|
+ replicaInfo.getBlockId(), true);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1749,6 +1756,10 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
if (asyncDiskService != null) {
|
|
|
asyncDiskService.shutdown();
|
|
|
}
|
|
|
+
|
|
|
+ if (asyncLazyPersistService != null) {
|
|
|
+ asyncLazyPersistService.shutdown();
|
|
|
+ }
|
|
|
|
|
|
if(volumes != null) {
|
|
|
volumes.shutdown();
|
|
@@ -2307,6 +2318,40 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
return new RollingLogsImpl(dir, prefix);
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public void onCompleteLazyPersist(String bpId, long blockId,
|
|
|
+ long creationTime, File[] savedFiles, FsVolumeImpl targetVolume) {
|
|
|
+ synchronized (FsDatasetImpl.this) {
|
|
|
+ ramDiskReplicaTracker.recordEndLazyPersist(bpId, blockId, savedFiles);
|
|
|
+
|
|
|
+ targetVolume.incDfsUsed(bpId,
|
|
|
+ savedFiles[0].length() + savedFiles[1].length());
|
|
|
+
|
|
|
+ // Update metrics (ignore the metadata file size)
|
|
|
+ datanode.getMetrics().incrRamDiskBlocksLazyPersisted();
|
|
|
+ datanode.getMetrics().incrRamDiskBytesLazyPersisted(savedFiles[1].length());
|
|
|
+ datanode.getMetrics().addRamDiskBlocksLazyPersistWindowMs(
|
|
|
+ Time.monotonicNow() - creationTime);
|
|
|
+
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("LazyWriter: Finish persisting RamDisk block: "
|
|
|
+ + " block pool Id: " + bpId + " block id: " + blockId
|
|
|
+ + " to block file " + savedFiles[1] + " and meta file " + savedFiles[0]
|
|
|
+ + " on target volume " + targetVolume);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onFailLazyPersist(String bpId, long blockId) {
|
|
|
+ RamDiskReplica block = null;
|
|
|
+ block = ramDiskReplicaTracker.getReplica(bpId, blockId);
|
|
|
+ if (block != null) {
|
|
|
+ LOG.warn("Failed to save replica " + block + ". re-enqueueing it.");
|
|
|
+ ramDiskReplicaTracker.reenqueueReplicaNotPersisted(block);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
|
|
|
FileDescriptor fd, long offset, long nbytes, int flags) {
|
|
@@ -2315,9 +2360,38 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
nbytes, flags);
|
|
|
}
|
|
|
|
|
|
- void discardRamDiskReplica(RamDiskReplica replica, boolean deleteSavedCopies) {
|
|
|
- ramDiskReplicaTracker.discardReplica(replica.getBlockPoolId(),
|
|
|
- replica.getBlockId(), deleteSavedCopies);
|
|
|
+ private boolean ramDiskConfigured() {
|
|
|
+ for (FsVolumeImpl v: getVolumes()){
|
|
|
+ if (v.isTransientStorage()) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Add/Remove per DISK volume async lazy persist thread when RamDisk volume is
|
|
|
+ // added or removed.
|
|
|
+ // This should only be called when the FsDataSetImpl#volumes list is finalized.
|
|
|
+ private void setupAsyncLazyPersistThreads() {
|
|
|
+ boolean ramDiskConfigured = ramDiskConfigured();
|
|
|
+ for (FsVolumeImpl v: getVolumes()){
|
|
|
+ // Skip transient volumes
|
|
|
+ if (v.isTransientStorage()) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Add thread for DISK volume if RamDisk is configured
|
|
|
+ if (ramDiskConfigured &&
|
|
|
+ !asyncLazyPersistService.queryVolume(v.getCurrentDir())) {
|
|
|
+ asyncLazyPersistService.addVolume(v.getCurrentDir());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Remove thread for DISK volume if RamDisk is not configured
|
|
|
+ if (!ramDiskConfigured &&
|
|
|
+ asyncLazyPersistService.queryVolume(v.getCurrentDir())) {
|
|
|
+ asyncLazyPersistService.removeVolume(v.getCurrentDir());
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
class LazyWriter implements Runnable {
|
|
@@ -2343,61 +2417,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
DFSConfigKeys.DFS_DATANODE_RAM_DISK_LOW_WATERMARK_REPLICAS_DEFAULT);
|
|
|
}
|
|
|
|
|
|
- private void moveReplicaToNewVolume(String bpid, long blockId, long creationTime)
|
|
|
- throws IOException {
|
|
|
-
|
|
|
- FsVolumeImpl targetVolume;
|
|
|
- ReplicaInfo replicaInfo;
|
|
|
- BlockPoolSlice bpSlice;
|
|
|
- File srcFile, srcMeta;
|
|
|
- long genStamp;
|
|
|
-
|
|
|
- synchronized (FsDatasetImpl.this) {
|
|
|
- replicaInfo = volumeMap.get(bpid, blockId);
|
|
|
-
|
|
|
- if (replicaInfo == null || !replicaInfo.getVolume().isTransientStorage()) {
|
|
|
- // The block was either deleted before it could be checkpointed or
|
|
|
- // it is already on persistent storage. This can occur if a second
|
|
|
- // replica on persistent storage was found after the lazy write was
|
|
|
- // scheduled.
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- // Pick a target volume for the block.
|
|
|
- targetVolume = volumes.getNextVolume(
|
|
|
- StorageType.DEFAULT, replicaInfo.getNumBytes());
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("LazyWriter starting to save blockId=" + blockId + "; bpid=" + bpid);
|
|
|
- }
|
|
|
-
|
|
|
- ramDiskReplicaTracker.recordStartLazyPersist(bpid, blockId, targetVolume);
|
|
|
- bpSlice = targetVolume.getBlockPoolSlice(bpid);
|
|
|
- srcMeta = replicaInfo.getMetaFile();
|
|
|
- srcFile = replicaInfo.getBlockFile();
|
|
|
- genStamp = replicaInfo.getGenerationStamp();
|
|
|
- }
|
|
|
-
|
|
|
- // Drop the FsDatasetImpl lock for the file copy.
|
|
|
- File[] savedFiles =
|
|
|
- bpSlice.lazyPersistReplica(blockId, genStamp, srcMeta, srcFile);
|
|
|
-
|
|
|
- synchronized (FsDatasetImpl.this) {
|
|
|
- ramDiskReplicaTracker.recordEndLazyPersist(bpid, blockId, savedFiles);
|
|
|
-
|
|
|
- // Update metrics (ignore the metadata file size)
|
|
|
- datanode.getMetrics().incrRamDiskBlocksLazyPersisted();
|
|
|
- datanode.getMetrics().incrRamDiskBytesLazyPersisted(replicaInfo.getNumBytes());
|
|
|
- datanode.getMetrics().addRamDiskBlocksLazyPersistWindowMs(
|
|
|
- Time.monotonicNow() - creationTime);
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("LazyWriter finished saving blockId=" + blockId + "; bpid=" + bpid +
|
|
|
- " to file " + savedFiles[1]);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Checkpoint a pending replica to persistent storage now.
|
|
|
* If we fail then move the replica to the end of the queue.
|
|
@@ -2405,13 +2424,43 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
*/
|
|
|
private boolean saveNextReplica() {
|
|
|
RamDiskReplica block = null;
|
|
|
+ FsVolumeImpl targetVolume;
|
|
|
+ ReplicaInfo replicaInfo;
|
|
|
boolean succeeded = false;
|
|
|
|
|
|
try {
|
|
|
block = ramDiskReplicaTracker.dequeueNextReplicaToPersist();
|
|
|
if (block != null) {
|
|
|
- moveReplicaToNewVolume(block.getBlockPoolId(), block.getBlockId(),
|
|
|
- block.getCreationTime());
|
|
|
+ synchronized (FsDatasetImpl.this) {
|
|
|
+ replicaInfo = volumeMap.get(block.getBlockPoolId(), block.getBlockId());
|
|
|
+
|
|
|
+ // If replicaInfo is null, the block was either deleted before
|
|
|
+ // it could be checkpointed or it is already on persistent storage.
|
|
|
+ // This can occur if a second replica on persistent storage was found
|
|
|
+ // after the lazy write was scheduled.
|
|
|
+ if (replicaInfo != null &&
|
|
|
+ replicaInfo.getVolume().isTransientStorage()) {
|
|
|
+ // Pick a target volume to persist the block.
|
|
|
+ targetVolume = volumes.getNextVolume(
|
|
|
+ StorageType.DEFAULT, replicaInfo.getNumBytes());
|
|
|
+
|
|
|
+ ramDiskReplicaTracker.recordStartLazyPersist(
|
|
|
+ block.getBlockPoolId(), block.getBlockId(), targetVolume);
|
|
|
+
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("LazyWriter: Start persisting RamDisk block:"
|
|
|
+ + " block pool Id: " + block.getBlockPoolId()
|
|
|
+ + " block id: " + block.getBlockId()
|
|
|
+ + " on target volume " + targetVolume);
|
|
|
+ }
|
|
|
+
|
|
|
+ asyncLazyPersistService.submitLazyPersistTask(
|
|
|
+ block.getBlockPoolId(), block.getBlockId(),
|
|
|
+ replicaInfo.getGenerationStamp(), block.getCreationTime(),
|
|
|
+ replicaInfo.getMetaFile(), replicaInfo.getBlockFile(),
|
|
|
+ targetVolume);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
succeeded = true;
|
|
|
} catch(IOException ioe) {
|
|
@@ -2419,10 +2468,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
} finally {
|
|
|
if (!succeeded && block != null) {
|
|
|
LOG.warn("Failed to save replica " + block + ". re-enqueueing it.");
|
|
|
- ramDiskReplicaTracker.reenqueueReplicaNotPersisted(block);
|
|
|
+ onFailLazyPersist(block.getBlockPoolId(), block.getBlockId());
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
return succeeded;
|
|
|
}
|
|
|
|
|
@@ -2479,7 +2527,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
metaFile = replicaInfo.getMetaFile();
|
|
|
blockFileUsed = blockFile.length();
|
|
|
metaFileUsed = metaFile.length();
|
|
|
- discardRamDiskReplica(replicaState, false);
|
|
|
+ ramDiskReplicaTracker.discardReplica(replicaState.getBlockPoolId(),
|
|
|
+ replicaState.getBlockId(), false);
|
|
|
|
|
|
// Move the replica from lazyPersist/ to finalized/ on target volume
|
|
|
BlockPoolSlice bpSlice =
|