Quellcode durchsuchen

HDFS-10694. processReport() should print blockReportId in each log message. Contributed by Yuanbo Liu.

(cherry picked from commit 10e84c6a6e831fe2bea061fb21bd0dfe32bc9953)
Yuanbo Liu vor 9 Jahren
Ursprung
Commit
44162e555a

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

@@ -1954,6 +1954,8 @@ public class BlockManager implements BlockStatsMXBean {
     final long endTime;
     DatanodeDescriptor node;
     Collection<Block> invalidatedBlocks = Collections.emptyList();
+    String strBlockReportId =
+        context != null ? Long.toHexString(context.getReportId()) : "";
 
     try {
       node = datanodeManager.getDatanode(nodeID);
@@ -1972,9 +1974,10 @@ public class BlockManager implements BlockStatsMXBean {
       }
       if (namesystem.isInStartupSafeMode()
           && storageInfo.getBlockReportCount() > 0) {
-        blockLog.info("BLOCK* processReport: "
+        blockLog.info("BLOCK* processReport 0x{}: "
             + "discarded non-initial block report from {}"
-            + " because namenode still in startup phase", nodeID);
+            + " because namenode still in startup phase",
+            strBlockReportId, nodeID);
         blockReportLeaseManager.removeLease(node);
         return !node.hasStaleStorages();
       }
@@ -1988,12 +1991,14 @@ public class BlockManager implements BlockStatsMXBean {
       if (storageInfo.getBlockReportCount() == 0) {
         // The first block report can be processed a lot more efficiently than
         // ordinary block reports.  This shortens restart times.
-        LOG.info("Processing first storage report for " +
-            storageInfo.getStorageID() + " from datanode " +
+        blockLog.info("BLOCK* processReport 0x{}: Processing first "
+                + "storage report for {} from datanode {}",
+            strBlockReportId,
+            storageInfo.getStorageID(),
             nodeID.getDatanodeUuid());
         processFirstBlockReport(storageInfo, newReport);
       } else {
-        invalidatedBlocks = processReport(storageInfo, newReport);
+        invalidatedBlocks = processReport(storageInfo, newReport, context);
       }
       
       storageInfo.receivedBlockReport();
@@ -2029,8 +2034,8 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     for (Block b : invalidatedBlocks) {
-      blockLog.debug("BLOCK* processReport: {} on node {} size {} does not " +
-          "belong to any file", b, node, b.getNumBytes());
+      blockLog.debug("BLOCK* processReport 0x{}: {} on node {} size {} does not"
+          + " belong to any file", strBlockReportId, b, node, b.getNumBytes());
     }
 
     // Log the block report processing stats from Namenode perspective
@@ -2038,10 +2043,10 @@ public class BlockManager implements BlockStatsMXBean {
     if (metrics != null) {
       metrics.addBlockReport((int) (endTime - startTime));
     }
-    blockLog.info("BLOCK* processReport: from storage {} node {}, " +
+    blockLog.info("BLOCK* processReport 0x{}: from storage {} node {}, " +
         "blocks: {}, hasStaleStorage: {}, processing time: {} msecs, " +
-        "invalidatedBlocks: {}", storage.getStorageID(), nodeID,
-        newReport.getNumberOfBlocks(),
+        "invalidatedBlocks: {}", strBlockReportId, storage.getStorageID(),
+        nodeID, newReport.getNumberOfBlocks(),
         node.hasStaleStorages(), (endTime - startTime),
         invalidatedBlocks.size());
     return !node.hasStaleStorages();
@@ -2149,7 +2154,8 @@ public class BlockManager implements BlockStatsMXBean {
   
   private Collection<Block> processReport(
       final DatanodeStorageInfo storageInfo,
-      final BlockListAsLongs report) throws IOException {
+      final BlockListAsLongs report,
+      BlockReportContext context) throws IOException {
     // Normal case:
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
@@ -2161,6 +2167,11 @@ public class BlockManager implements BlockStatsMXBean {
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
     reportDiff(storageInfo, report,
         toAdd, toRemove, toInvalidate, toCorrupt, toUC);
+
+    String strBlockReportId = "";
+    if (context != null) {
+      strBlockReportId = Long.toHexString(context.getReportId());
+    }
    
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     // Process the blocks on each queue
@@ -2176,8 +2187,8 @@ public class BlockManager implements BlockStatsMXBean {
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
-      blockLog.info("BLOCK* processReport: logged info for {} of {} " +
-          "reported.", maxNumBlocksToLog, numBlocksLogged);
+      blockLog.info("BLOCK* processReport 0x{}: logged info for {} of {} " +
+          "reported.", strBlockReportId, maxNumBlocksToLog, numBlocksLogged);
     }
     for (Block b : toInvalidate) {
       addToInvalidates(b, node);