|
@@ -2305,10 +2305,10 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
throws IOException {
|
|
|
final List<String> errors = new ArrayList<String>();
|
|
|
for (int i = 0; i < invalidBlks.length; i++) {
|
|
|
- final ReplicaInfo removing;
|
|
|
+ final ReplicaInfo info;
|
|
|
final FsVolumeImpl v;
|
|
|
- try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) {
|
|
|
- final ReplicaInfo info = volumeMap.get(bpid, invalidBlks[i]);
|
|
|
+ try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) {
|
|
|
+ info = volumeMap.get(bpid, invalidBlks[i]);
|
|
|
if (info == null) {
|
|
|
ReplicaInfo infoByBlockId =
|
|
|
volumeMap.get(bpid, invalidBlks[i].getBlockId());
|
|
@@ -2342,48 +2342,21 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
LOG.warn("Parent directory check failed; replica {} is " +
|
|
|
"not backed by a local file", info);
|
|
|
}
|
|
|
- removing = volumeMap.remove(bpid, invalidBlks[i]);
|
|
|
- addDeletingBlock(bpid, removing.getBlockId());
|
|
|
- LOG.debug("Block file {} is to be deleted", removing.getBlockURI());
|
|
|
- datanode.getMetrics().incrBlocksRemoved(1);
|
|
|
- if (removing instanceof ReplicaInPipeline) {
|
|
|
- ((ReplicaInPipeline) removing).releaseAllBytesReserved();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- if (v.isTransientStorage()) {
|
|
|
- RamDiskReplica replicaInfo =
|
|
|
- ramDiskReplicaTracker.getReplica(bpid, invalidBlks[i].getBlockId());
|
|
|
- if (replicaInfo != null) {
|
|
|
- if (!replicaInfo.getIsPersisted()) {
|
|
|
- datanode.getMetrics().incrRamDiskBlocksDeletedBeforeLazyPersisted();
|
|
|
- }
|
|
|
- ramDiskReplicaTracker.discardReplica(replicaInfo.getBlockPoolId(),
|
|
|
- replicaInfo.getBlockId(), true);
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
- // If a DFSClient has the replica in its cache of short-circuit file
|
|
|
- // descriptors (and the client is using ShortCircuitShm), invalidate it.
|
|
|
- datanode.getShortCircuitRegistry().processBlockInvalidation(
|
|
|
- new ExtendedBlockId(invalidBlks[i].getBlockId(), bpid));
|
|
|
-
|
|
|
- // If the block is cached, start uncaching it.
|
|
|
- cacheManager.uncacheBlock(bpid, invalidBlks[i].getBlockId());
|
|
|
-
|
|
|
try {
|
|
|
if (async) {
|
|
|
// Delete the block asynchronously to make sure we can do it fast
|
|
|
// enough.
|
|
|
// It's ok to unlink the block file before the uncache operation
|
|
|
// finishes.
|
|
|
- asyncDiskService.deleteAsync(v.obtainReference(), removing,
|
|
|
+ asyncDiskService.deleteAsync(v.obtainReference(), info,
|
|
|
new ExtendedBlock(bpid, invalidBlks[i]),
|
|
|
- dataStorage.getTrashDirectoryForReplica(bpid, removing));
|
|
|
+ dataStorage.getTrashDirectoryForReplica(bpid, info));
|
|
|
} else {
|
|
|
- asyncDiskService.deleteSync(v.obtainReference(), removing,
|
|
|
+ asyncDiskService.deleteSync(v.obtainReference(), info,
|
|
|
new ExtendedBlock(bpid, invalidBlks[i]),
|
|
|
- dataStorage.getTrashDirectoryForReplica(bpid, removing));
|
|
|
+ dataStorage.getTrashDirectoryForReplica(bpid, info));
|
|
|
}
|
|
|
} catch (ClosedChannelException e) {
|
|
|
LOG.warn("Volume {} is closed, ignore the deletion task for " +
|
|
@@ -2422,6 +2395,91 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
block.getStorageUuid());
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Remove Replica from ReplicaMap.
|
|
|
+ *
|
|
|
+ * @param block
|
|
|
+ * @param volume
|
|
|
+ * @return
|
|
|
+ */
|
|
|
+ boolean removeReplicaFromMem(final ExtendedBlock block, final FsVolumeImpl volume) {
|
|
|
+ final String bpid = block.getBlockPoolId();
|
|
|
+ final Block localBlock = block.getLocalBlock();
|
|
|
+ final long blockId = localBlock.getBlockId();
|
|
|
+ try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) {
|
|
|
+ final ReplicaInfo info = volumeMap.get(bpid, localBlock);
|
|
|
+ if (info == null) {
|
|
|
+ ReplicaInfo infoByBlockId = volumeMap.get(bpid, blockId);
|
|
|
+ if (infoByBlockId == null) {
|
|
|
+ // It is okay if the block is not found -- it
|
|
|
+ // may be deleted earlier.
|
|
|
+ LOG.info("Failed to delete replica {}: ReplicaInfo not found " +
|
|
|
+ "in removeReplicaFromMem.", localBlock);
|
|
|
+ } else {
|
|
|
+ LOG.error("Failed to delete replica {}: GenerationStamp not matched, " +
|
|
|
+ "existing replica is {} in removeReplicaFromMem.",
|
|
|
+ localBlock, Block.toString(infoByBlockId));
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ FsVolumeImpl v = (FsVolumeImpl) info.getVolume();
|
|
|
+ if (v == null) {
|
|
|
+ LOG.error("Failed to delete replica {}. No volume for this replica {} " +
|
|
|
+ "in removeReplicaFromMem.", localBlock, info);
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ File blockFile = new File(info.getBlockURI());
|
|
|
+ if (blockFile.getParentFile() == null) {
|
|
|
+ LOG.error("Failed to delete replica {}. Parent not found for block file: {} " +
|
|
|
+ "in removeReplicaFromMem.", localBlock, blockFile);
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ } catch(IllegalArgumentException e) {
|
|
|
+ LOG.warn("Parent directory check failed; replica {} is " +
|
|
|
+ "not backed by a local file in removeReplicaFromMem.", info);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!volume.getStorageID().equals(v.getStorageID())) {
|
|
|
+ LOG.error("Failed to delete replica {}. Appear different volumes, oldVolume: {} " +
|
|
|
+ "and newVolume: {} for this replica in removeReplicaFromMem.",
|
|
|
+ localBlock, volume, v);
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ ReplicaInfo removing = volumeMap.remove(bpid, localBlock);
|
|
|
+ addDeletingBlock(bpid, removing.getBlockId());
|
|
|
+ LOG.debug("Block file {} is to be deleted", removing.getBlockURI());
|
|
|
+ datanode.getMetrics().incrBlocksRemoved(1);
|
|
|
+ if (removing instanceof ReplicaInPipeline) {
|
|
|
+ ((ReplicaInPipeline) removing).releaseAllBytesReserved();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (volume.isTransientStorage()) {
|
|
|
+ RamDiskReplicaTracker.RamDiskReplica replicaInfo = ramDiskReplicaTracker.
|
|
|
+ getReplica(bpid, blockId);
|
|
|
+ if (replicaInfo != null) {
|
|
|
+ if (!replicaInfo.getIsPersisted()) {
|
|
|
+ datanode.getMetrics().incrRamDiskBlocksDeletedBeforeLazyPersisted();
|
|
|
+ }
|
|
|
+ ramDiskReplicaTracker.discardReplica(replicaInfo.getBlockPoolId(),
|
|
|
+ replicaInfo.getBlockId(), true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // If a DFSClient has the replica in its cache of short-circuit file
|
|
|
+ // descriptors (and the client is using ShortCircuitShm), invalidate it.
|
|
|
+ datanode.getShortCircuitRegistry().processBlockInvalidation(
|
|
|
+ ExtendedBlockId.fromExtendedBlock(block));
|
|
|
+
|
|
|
+ // If the block is cached, start uncaching it.
|
|
|
+ cacheManager.uncacheBlock(bpid, blockId);
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Asynchronously attempts to cache a single block via {@link FsDatasetCache}.
|
|
|
*/
|
|
@@ -3628,8 +3686,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- private void addDeletingBlock(String bpid, Long blockId) {
|
|
|
+
|
|
|
+ protected void addDeletingBlock(String bpid, Long blockId) {
|
|
|
synchronized(deletingBlock) {
|
|
|
Set<Long> s = deletingBlock.get(bpid);
|
|
|
if (s == null) {
|