瀏覽代碼

HDFS-12673. Ozone: Log elapsed time for block deletion tasks. Contributed by Yiqun Lin.

Weiwei Yang 7 年之前
父節點
當前提交
0f0417a12c

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java

@@ -201,7 +201,7 @@ public class BlockDeletingService extends BackgroundService{
             File chunkFile = dataDir.toPath()
                 .resolve(chunkInfo.getChunkName()).toFile();
             if (FileUtils.deleteQuietly(chunkFile)) {
-              LOG.info("block {} chunk {} deleted", blockName,
+              LOG.debug("block {} chunk {} deleted", blockName,
                   chunkFile.getAbsolutePath());
             }
           }

+ 10 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyDeletingService.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.scm.protocol.ScmBlockLocationProtocol;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.utils.BackgroundService;
 import org.apache.hadoop.utils.BackgroundTask;
 import org.apache.hadoop.utils.BackgroundTaskQueue;
@@ -91,6 +92,7 @@ public class KeyDeletingService extends BackgroundService {
     @Override
     public BackgroundTaskResult call() throws Exception {
       try {
+        long startTime = Time.monotonicNow();
         List<BlockGroup> keyBlocksList = manager
             .getPendingDeletionKeys(keyLimitPerTask);
         if (keyBlocksList.size() > 0) {
@@ -102,7 +104,7 @@ public class KeyDeletingService extends BackgroundService {
               try {
                 // Purge key from KSM DB.
                 manager.deletePendingDeletionKey(result.getObjectKey());
-                LOG.info("Key {} deleted from KSM DB", result.getObjectKey());
+                LOG.debug("Key {} deleted from KSM DB", result.getObjectKey());
               } catch (IOException e) {
                 // if a pending deletion key is failed to delete,
                 // print a warning here and retain it in this state,
@@ -118,6 +120,13 @@ public class KeyDeletingService extends BackgroundService {
                   String.join(",", result.getFailedBlocks()));
             }
           }
+
+          if (!results.isEmpty()) {
+            LOG.info("Number of key deleted from KSM DB: {},"
+                + " task elapsed time: {}ms",
+                results.size(), Time.monotonicNow() - startTime);
+          }
+
           return results::size;
         } else {
           LOG.debug("No pending deletion key found in KSM");

+ 17 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/SCMBlockDeletingService.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.ozone.protocol.proto.StorageContainerDatanodeProtocolPr
 import org.apache.hadoop.ozone.scm.container.Mapping;
 import org.apache.hadoop.ozone.scm.node.NodeManager;
 import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.utils.BackgroundService;
 import org.apache.hadoop.utils.BackgroundTask;
 import org.apache.hadoop.utils.BackgroundTaskQueue;
@@ -98,28 +99,39 @@ public class SCMBlockDeletingService extends BackgroundService {
 
     @Override
     public EmptyTaskResult call() throws Exception {
+      int dnTxCount = 0;
+      long startTime = Time.monotonicNow();
       // Scan SCM DB in HB interval and collect a throttled list of
       // to delete blocks.
       LOG.debug("Running DeletedBlockTransactionScanner");
       DatanodeDeletedBlockTransactions transactions =
           getToDeleteContainerBlocks();
       if (transactions != null && !transactions.isEmpty()) {
-        transactions.getDatanodes().forEach(datanodeID -> {
-          List<DeletedBlocksTransaction> dnTXs =
-              transactions.getDatanodeTransactions(datanodeID);
+        for (DatanodeID datanodeID : transactions.getDatanodes()) {
+          List<DeletedBlocksTransaction> dnTXs = transactions
+              .getDatanodeTransactions(datanodeID);
+          dnTxCount += dnTXs.size();
           // TODO commandQueue needs a cap.
           // We should stop caching new commands if num of un-processed
           // command is bigger than a limit, e.g 50. In case datanode goes
           // offline for sometime, the cached commands be flooded.
           nodeManager.addDatanodeCommand(datanodeID,
               new DeleteBlocksCommand(dnTXs));
-          LOG.info("Added delete block command for datanode {} in the queue,"
+          LOG.debug(
+              "Added delete block command for datanode {} in the queue,"
                   + " number of delete block transactions: {}, TxID list: {}",
               datanodeID, dnTXs.size(),
               String.join(",", transactions.getTransactionIDList(datanodeID)));
+        }
+      }
 
-        });
+      if (dnTxCount > 0) {
+        LOG.info("Totally added {} delete blocks command for"
+            + " {} datanodes, task elapsed time: {}ms",
+            dnTxCount, transactions.getDatanodes().size(),
+            Time.monotonicNow() - startTime);
       }
+
       return EmptyTaskResult.newResult();
     }