Browse Source

HDFS-12298. Ozone: Block deletion service floods the log when deleting large number of block files. Contributed by Yiqun Lin.

Weiwei Yang 8 years ago
parent
commit
e61530f5d9

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

@@ -28,6 +28,7 @@ import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
 import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.utils.BackgroundService;
 import org.apache.hadoop.utils.BackgroundTaskResult;
 import org.apache.hadoop.utils.BackgroundTaskQueue;
@@ -157,6 +158,7 @@ public class BlockDeletingService extends BackgroundService{
 
     @Override
     public BackgroundTaskResult call() throws Exception {
+      long startTime = Time.monotonicNow();
       // Scan container's db and get list of under deletion blocks
       MetadataStore meta = KeyUtils.getDB(containerData, conf);
       // # of blocks to delete is throttled
@@ -165,16 +167,16 @@ public class BlockDeletingService extends BackgroundService{
       List<Map.Entry<byte[], byte[]>> toDeleteBlocks =
           meta.getRangeKVs(null, blockLimitPerTask, filter);
       if (toDeleteBlocks.isEmpty()) {
-        LOG.info("No under deletion block found in container : {}",
+        LOG.debug("No under deletion block found in container : {}",
             containerData.getContainerName());
       }
 
       List<String> succeedBlocks = new LinkedList<>();
-      LOG.info("Container : {}, To-Delete blocks : {}",
+      LOG.debug("Container : {}, To-Delete blocks : {}",
           containerData.getContainerName(), toDeleteBlocks.size());
       toDeleteBlocks.forEach(entry -> {
         String blockName = DFSUtil.bytes2String(entry.getKey());
-        LOG.info("Deleting block {}", blockName);
+        LOG.debug("Deleting block {}", blockName);
         try {
           ContainerProtos.KeyData data =
               ContainerProtos.KeyData.parseFrom(entry.getValue());
@@ -182,7 +184,7 @@ public class BlockDeletingService extends BackgroundService{
           for (ContainerProtos.ChunkInfo chunkInfo : data.getChunksList()) {
             File chunkFile = new File(chunkInfo.getChunkName());
             if (FileUtils.deleteQuietly(chunkFile)) {
-              LOG.info("block {} chunk {} deleted", blockName,
+              LOG.debug("block {} chunk {} deleted", blockName,
                   chunkFile.getAbsolutePath());
             }
           }
@@ -198,6 +200,10 @@ public class BlockDeletingService extends BackgroundService{
           batch.delete(DFSUtil.string2Bytes(entry)));
       meta.writeBatch(batch);
 
+      LOG.info("The elapsed time of task@{} for"
+          + " deleting blocks: {}ms.",
+          Integer.toHexString(this.hashCode()),
+          Time.monotonicNow() - startTime);
       ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult();
       crr.addAll(succeedBlocks);
       return crr;