Ver código fonte

HDFS-6860. BlockStateChange logs are too noisy. Contributed by Chang Li and Xiaoyu Yao.

(cherry picked from commit d311a38a6b32bbb210bd8748cfb65463e9c0740e)
Xiaoyu Yao 9 anos atrás
pai
commit
52ee3ecdf6

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -425,6 +425,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-7192. DN should ignore lazyPersist hint if the writer is
     not local. (Arpit Agarwal)
 
+    HDFS-6860. BlockStateChange logs are too noisy. (Chang Li and xyao via xyao)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
@@ -237,7 +236,7 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
     for (ReplicaUnderConstruction r : replicas) {
       if (genStamp != r.getGenerationStamp()) {
         r.getExpectedStorageLocation().removeBlock(this);
-        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
+        NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica "
             + "from location: {}", r.getExpectedStorageLocation());
       }
     }
@@ -310,7 +309,7 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
       primary.getExpectedStorageLocation().
           getDatanodeDescriptor().addBlockToBeRecovered(this);
       primary.setChosenAsPrimary(true);
-      NameNode.blockStateChangeLog.info(
+      NameNode.blockStateChangeLog.debug(
           "BLOCK* {} recovery started, primary={}", this, primary);
     }
   }

+ 19 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -1142,7 +1142,7 @@ public class BlockManager implements BlockStatsMXBean {
       datanodes.append(node).append(" ");
     }
     if (datanodes.length() != 0) {
-      blockLog.info("BLOCK* addToInvalidates: {} {}", storedBlock,
+      blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock,
           datanodes.toString());
     }
   }
@@ -1177,7 +1177,7 @@ public class BlockManager implements BlockStatsMXBean {
       // ignore the request for now. This could happen when BlockScanner
       // thread of Datanode reports bad block before Block reports are sent
       // by the Datanode on startup
-      blockLog.info("BLOCK* findAndMarkBlockAsCorrupt: {} not found", blk);
+      blockLog.debug("BLOCK* findAndMarkBlockAsCorrupt: {} not found", blk);
       return;
     }
 
@@ -1205,7 +1205,7 @@ public class BlockManager implements BlockStatsMXBean {
       DatanodeDescriptor node) throws IOException {
 
     if (b.stored.isDeleted()) {
-      blockLog.info("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
+      blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
           " corrupt as it does not belong to any file", b);
       addToInvalidates(b.corrupted, node);
       return;
@@ -1255,7 +1255,7 @@ public class BlockManager implements BlockStatsMXBean {
    */
   private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn
       ) throws IOException {
-    blockLog.info("BLOCK* invalidateBlock: {} on {}", b, dn);
+    blockLog.debug("BLOCK* invalidateBlock: {} on {}", b, dn);
     DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
     if (node == null) {
       throw new IOException("Cannot invalidate " + b
@@ -1265,7 +1265,7 @@ public class BlockManager implements BlockStatsMXBean {
     // Check how many copies we have of the block
     NumberReplicas nr = countNodes(b.stored);
     if (nr.replicasOnStaleNodes() > 0) {
-      blockLog.info("BLOCK* invalidateBlocks: postponing " +
+      blockLog.debug("BLOCK* invalidateBlocks: postponing " +
           "invalidation of {} on {} because {} replica(s) are located on " +
           "nodes with potentially out-of-date block reports", b, dn,
           nr.replicasOnStaleNodes());
@@ -1279,7 +1279,7 @@ public class BlockManager implements BlockStatsMXBean {
           b, dn);
       return true;
     } else {
-      blockLog.info("BLOCK* invalidateBlocks: {} on {} is the only copy and" +
+      blockLog.debug("BLOCK* invalidateBlocks: {} on {} is the only copy and" +
           " was not deleted", b, dn);
       return false;
     }
@@ -1410,7 +1410,7 @@ public class BlockManager implements BlockStatsMXBean {
               if ( (pendingReplications.getNumReplicas(block) > 0) ||
                    (blockHasEnoughRacks(block, requiredReplication)) ) {
                 neededReplications.remove(block, priority); // remove from neededReplications
-                blockLog.info("BLOCK* Removing {} from neededReplications as" +
+                blockLog.debug("BLOCK* Removing {} from neededReplications as" +
                         " it has enough replicas", block);
                 continue;
               }
@@ -1480,7 +1480,7 @@ public class BlockManager implements BlockStatsMXBean {
                  (blockHasEnoughRacks(block, requiredReplication)) ) {
               neededReplications.remove(block, priority); // remove from neededReplications
               rw.targets = null;
-              blockLog.info("BLOCK* Removing {} from neededReplications as" +
+              blockLog.debug("BLOCK* Removing {} from neededReplications as" +
                       " it has enough replicas", block);
               continue;
             }
@@ -1528,7 +1528,7 @@ public class BlockManager implements BlockStatsMXBean {
             targetList.append(' ');
             targetList.append(targets[k].getDatanodeDescriptor());
           }
-          blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNode,
+          blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.srcNode,
               rw.block, targetList);
         }
       }
@@ -2130,7 +2130,7 @@ public class BlockManager implements BlockStatsMXBean {
         }
       }
       if (isCorrupt) {
-        blockLog.info("BLOCK* markBlockReplicasAsCorrupt: mark block replica" +
+        blockLog.debug("BLOCK* markBlockReplicasAsCorrupt: mark block replica" +
             " {} on {} as corrupt because the dn is not in the new committed " +
             "storage list.", b, storage.getDatanodeDescriptor());
         markBlockAsCorrupt(b, storage, storage.getDatanodeDescriptor());
@@ -2625,7 +2625,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
     if (storedBlock == null || storedBlock.isDeleted()) {
       // If this block does not belong to anyfile, then we are done.
-      blockLog.info("BLOCK* addStoredBlock: {} on {} size {} but it does not" +
+      blockLog.debug("BLOCK* addStoredBlock: {} on {} size {} but it does not" +
           " belong to any file", block, node, block.getNumBytes());
 
       // we could add this block to invalidate set of this datanode.
@@ -2715,10 +2715,10 @@ public class BlockManager implements BlockStatsMXBean {
 
   private void logAddStoredBlock(BlockInfo storedBlock,
       DatanodeDescriptor node) {
-    if (!blockLog.isInfoEnabled()) {
+    if (!blockLog.isDebugEnabled()) {
       return;
     }
-    
+
     StringBuilder sb = new StringBuilder(500);
     sb.append("BLOCK* addStoredBlock: blockMap updated: ")
       .append(node)
@@ -2726,7 +2726,7 @@ public class BlockManager implements BlockStatsMXBean {
     storedBlock.appendStringTo(sb);
     sb.append(" size " )
       .append(storedBlock.getNumBytes());
-    blockLog.info(sb.toString());
+    blockLog.debug(sb.toString());
   }
   /**
    * Invalidate corrupt replicas.
@@ -2756,7 +2756,7 @@ public class BlockManager implements BlockStatsMXBean {
           removedFromBlocksMap = false;
         }
       } catch (IOException e) {
-        blockLog.info("invalidateCorruptReplicas error in deleting bad block"
+        blockLog.debug("invalidateCorruptReplicas error in deleting bad block"
             + " {} on {}", blk, node, e);
         removedFromBlocksMap = false;
       }
@@ -3095,7 +3095,7 @@ public class BlockManager implements BlockStatsMXBean {
     // upon giving instructions to the datanodes.
     //
     addToInvalidates(storedBlock, chosen.getDatanodeDescriptor());
-    blockLog.info("BLOCK* chooseExcessReplicates: "
+    blockLog.debug("BLOCK* chooseExcessReplicates: "
         +"({}, {}) is added to invalidated blocks set", chosen, storedBlock);
   }
 
@@ -3293,11 +3293,11 @@ public class BlockManager implements BlockStatsMXBean {
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
-      blockLog.info("BLOCK* addBlock: logged info for {} of {} reported.",
+      blockLog.debug("BLOCK* addBlock: logged info for {} of {} reported.",
           maxNumBlocksToLog, numBlocksLogged);
     }
     for (Block b : toInvalidate) {
-      blockLog.info("BLOCK* addBlock: block {} on node {} size {} does not " +
+      blockLog.debug("BLOCK* addBlock: block {} on node {} size {} does not " +
           "belong to any file", b, node, b.getNumBytes());
       addToInvalidates(b, node);
     }
@@ -3643,7 +3643,7 @@ public class BlockManager implements BlockStatsMXBean {
     } finally {
       namesystem.writeUnlock();
     }
-    blockLog.info("BLOCK* {}: ask {} to delete {}", getClass().getSimpleName(),
+    blockLog.debug("BLOCK* {}: ask {} to delete {}", getClass().getSimpleName(),
         dn, toInvalidate);
     return toInvalidate.size();
   }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java

@@ -80,12 +80,12 @@ public class CorruptReplicasMap{
     }
     
     if (!nodes.keySet().contains(dn)) {
-      NameNode.blockStateChangeLog.info(
+      NameNode.blockStateChangeLog.debug(
           "BLOCK NameSystem.addToCorruptReplicasMap: {} added as corrupt on "
               + "{} by {} {}", blk.getBlockName(), dn, Server.getRemoteIp(),
           reasonText);
     } else {
-      NameNode.blockStateChangeLog.info(
+      NameNode.blockStateChangeLog.debug(
           "BLOCK NameSystem.addToCorruptReplicasMap: duplicate requested for" +
               " {} to add as corrupt on {} by {} {}", blk.getBlockName(), dn,
               Server.getRemoteIp(), reasonText);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java

@@ -113,7 +113,7 @@ class InvalidateBlocks {
     if (set.add(block)) {
       numBlocks++;
       if (log) {
-        NameNode.blockStateChangeLog.info("BLOCK* {}: add {} to {}",
+        NameNode.blockStateChangeLog.debug("BLOCK* {}: add {} to {}",
             getClass().getSimpleName(), block, datanode);
       }
     }