[GitHub] [hadoop] ZanderXu commented on a diff in pull request #4903: HDFS-16774.Improve async delete replica on datanode

2022-09-27 Thread GitBox


ZanderXu commented on code in PR #4903:
URL: https://github.com/apache/hadoop/pull/4903#discussion_r981060797


##
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java:
##
@@ -1891,23 +1894,22 @@ public void delayDeleteReplica() {
   // If this replica is deleted from memory, the client would got an 
ReplicaNotFoundException.
   assertNotNull(ds.getStoredBlock(bpid, extendedBlock.getBlockId()));
 
-  // Make it resume the removeReplicaFromMem method
+  // Make it resume the removeReplicaFromMem method.
   semaphore.release(1);
 
   // Sleep for 1 second so that datanode can complete invalidate.

Review Comment:
   How about change this comment to `Waiting for the async deletion task 
finish`?



##
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java:
##
@@ -1891,23 +1894,22 @@ public void delayDeleteReplica() {
   // If this replica is deleted from memory, the client would got an 
ReplicaNotFoundException.
   assertNotNull(ds.getStoredBlock(bpid, extendedBlock.getBlockId()));
 
-  // Make it resume the removeReplicaFromMem method
+  // Make it resume the removeReplicaFromMem method.
   semaphore.release(1);
 
   // Sleep for 1 second so that datanode can complete invalidate.
-  GenericTestUtils.waitFor(new com.google.common.base.Supplier() {
-@Override public Boolean get() {
-  return ds.asyncDiskService.countPendingDeletions() == 0;
-}
-  }, 100, 1000);
+  GenericTestUtils.waitFor(() -> 
ds.asyncDiskService.countPendingDeletions() == 0,
+  100, 1000);

Review Comment:
   single line.



##
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java:
##
@@ -2400,6 +2400,93 @@ public void invalidate(String bpid, ReplicaInfo block) {
 block.getStorageUuid());
   }
 
+  /**
+   * Remove Replica from ReplicaMap
+   *
+   * @param block
+   * @param volume
+   * @return
+   */
+  public boolean removeReplicaFromMem(final ExtendedBlock block, final 
FsVolumeImpl volume) {
+final String blockPoolId = block.getBlockPoolId();
+final Block localBlock = block.getLocalBlock();
+final long blockId = localBlock.getBlockId();
+try (AutoCloseableLock lock = lockManager.writeLock(
+LockLevel.BLOCK_POOl, blockPoolId)) {
+  final ReplicaInfo info = volumeMap.get(blockPoolId, localBlock);
+  if (info == null) {
+ReplicaInfo infoByBlockId =
+volumeMap.get(blockPoolId, 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();

Review Comment:
   `FsVolumeImpl v = (FsVolumeImpl) info.getVolume();`



##
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java:
##
@@ -2400,6 +2400,93 @@ public void invalidate(String bpid, ReplicaInfo block) {
 block.getStorageUuid());
   }
 
+  /**
+   * Remove Replica from ReplicaMap
+   *
+   * @param block
+   * @param volume
+   * @return
+   */
+  public boolean removeReplicaFromMem(final ExtendedBlock block, final 
FsVolumeImpl volume) {
+final String blockPoolId = block.getBlockPoolId();
+final Block localBlock = block.getLocalBlock();
+final long blockId = localBlock.getBlockId();
+try (AutoCloseableLock lock = lockManager.writeLock(
+LockLevel.BLOCK_POOl, blockPoolId)) {
+  final ReplicaInfo info = volumeMap.get(blockPoolId, localBlock);
+  if (info == null) {
+ReplicaInfo infoByBlockId =
+volumeMap.get(blockPoolId, 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 {} "
+  

[GitHub] [hadoop] ZanderXu commented on a diff in pull request #4903: HDFS-16774.Improve async delete replica on datanode

2022-09-20 Thread GitBox


ZanderXu commented on code in PR #4903:
URL: https://github.com/apache/hadoop/pull/4903#discussion_r975921564


##
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java:
##
@@ -2310,10 +2310,10 @@ private void invalidate(String bpid, Block[] 
invalidBlks, boolean async)
   throws IOException {
 final List errors = new ArrayList();
 for (int i = 0; i < invalidBlks.length; i++) {
-  final ReplicaInfo removing;
+  final ReplicaInfo info;

Review Comment:
   How about just move the IO request out of the lock?
   ```
   try {
 File blockFile = new File(info.getBlockURI());
 if (blockFile != null && blockFile.getParentFile() == null) {
   errors.add("Failed to delete replica " + invalidBlks[i]
   +  ". Parent not found for block file: " + blockFile);
   continue;
 }
   } catch(IllegalArgumentException e) {
 LOG.warn("Parent directory check failed; replica {} is " +
 "not backed by a local file", info);
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] ZanderXu commented on a diff in pull request #4903: HDFS-16774.Improve async delete replica on datanode

2022-09-19 Thread GitBox


ZanderXu commented on code in PR #4903:
URL: https://github.com/apache/hadoop/pull/4903#discussion_r974787840


##
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java:
##
@@ -359,6 +371,89 @@ public void run() {
 IOUtils.cleanupWithLogger(null, this.volumeRef);
   }
 }
+
+private boolean removeReplicaFromMem() {
+  try (AutoCloseableLock lock = 
fsdatasetImpl.acquireDatasetLockManager().writeLock(
+  DataNodeLockManager.LockLevel.BLOCK_POOl, block.getBlockPoolId())) {
+final ReplicaInfo info = fsdatasetImpl.volumeMap
+.get(block.getBlockPoolId(), block.getLocalBlock());
+if (info == null) {
+  ReplicaInfo infoByBlockId =
+  fsdatasetImpl.volumeMap.get(block.getBlockPoolId(),
+  block.getLocalBlock().getBlockId());
+  if (infoByBlockId == null) {
+// It is okay if the block is not found -- it
+// may be deleted earlier.
+LOG.info("Failed to delete replica " + block.getLocalBlock()
++ ": ReplicaInfo not found in removeReplicaFromMem.");

Review Comment:
   Can use `LOG.info("...{}...", block.getLocalBlock())`



##
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java:
##
@@ -359,6 +371,89 @@ public void run() {
 IOUtils.cleanupWithLogger(null, this.volumeRef);
   }
 }
+
+private boolean removeReplicaFromMem() {
+  try (AutoCloseableLock lock = 
fsdatasetImpl.acquireDatasetLockManager().writeLock(
+  DataNodeLockManager.LockLevel.BLOCK_POOl, block.getBlockPoolId())) {
+final ReplicaInfo info = fsdatasetImpl.volumeMap
+.get(block.getBlockPoolId(), block.getLocalBlock());
+if (info == null) {
+  ReplicaInfo infoByBlockId =
+  fsdatasetImpl.volumeMap.get(block.getBlockPoolId(),
+  block.getLocalBlock().getBlockId());
+  if (infoByBlockId == null) {
+// It is okay if the block is not found -- it
+// may be deleted earlier.
+LOG.info("Failed to delete replica " + block.getLocalBlock()
++ ": ReplicaInfo not found in removeReplicaFromMem.");
+  } else {
+LOG.error("Failed to delete replica " + block.getLocalBlock()
++ ": GenerationStamp not matched, existing replica is "
++ Block.toString(infoByBlockId) + " in removeReplicaFromMem.");
+  }
+  return false;
+}
+
+FsVolumeImpl v = (FsVolumeImpl)info.getVolume();
+if (v == null) {
+  LOG.error("Failed to delete replica " + block.getLocalBlock()
+  +  ". No volume for this replica " + info + " in 
removeReplicaFromMem.");
+  return false;
+}
+
+try {
+  File blockFile = new File(info.getBlockURI());
+  if (blockFile != null && blockFile.getParentFile() == null) {
+LOG.error("Failed to delete replica " + block.getLocalBlock()
++  ". Parent not found for block file: " + blockFile
++ " in removeReplicaFromMem.");
+return false;
+  }
+} catch(IllegalArgumentException e) {
+  LOG.warn("Parent directory check failed; replica {} is " +
+  "not backed by a local file in removeReplicaFromMem.", info);
+}
+
+if (!this.volume.getStorageID().equals(v.getStorageID())) {
+  LOG.error("Failed to delete replica " + block.getLocalBlock()
+  +  ". Appear different volumes, oldVolume=" + this.volume + " 
and newVolume=" + v
+  +  " for this replica in removeReplicaFromMem.");

Review Comment:
   Add here



##
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java:
##
@@ -1830,4 +1833,92 @@ public void testTransferAndNativeCopyMetrics() throws 
IOException {
   assertEquals(3, metrics.getNativeCopyIoQuantiles().length);
 }
   }
+
+  @Test
+  public void testAysncDiskServiceDeleteReplica()
+  throws IOException, InterruptedException, TimeoutException {
+HdfsConfiguration conf = new HdfsConfiguration();
+// Bump up replication interval.
+conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 
10);
+MiniDFSCluster cluster =
+new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+DistributedFileSystem fs = cluster.getFileSystem();
+String bpid = cluster.getNamesystem().getBlockPoolId();
+DataNodeFaultInjector oldInjector = DataNodeFaultInjector.get();
+final Semaphore semaphore = new Semaphore(0);
+try {
+  cluster.waitActive();
+  final DataNodeFaultInjector injector = new DataNodeFaultInjector() {
+@Override
+public void