Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-17453

IncrementalBlockReport can have race condition with Edit Log Tailer

    XMLWordPrintableJSON

Details

    • Reviewed

    Description

      Summary

      There is a race condition between IncrementalBlockReports (IBR) and EditLogTailer in Standby NameNode (SNN) which can lead to leaked IBRs and false corrupt blocks after HA Failover. The race condition occurs when the SNN loads the edit logs before it receives the block reports from DataNode (DN).

      Example

      In the following example there is a block (b1) with 3 generation stamps (gs1, gs2, gs3).

      1. SNN1 loads edit logs for b1gs1 and b1gs2.
      2. DN1 sends the IBR for b1gs1 to SNN1.
      3. SNN1 will determine that the reported block b1gs1 from DN1 is corrupt and it will be queued for later. BlockManager.java
            BlockToMarkCorrupt c = checkReplicaCorrupt(
                block, reportedState, storedBlock, ucState, dn);
            if (c != null) {
              if (shouldPostponeBlocksFromFuture) {
                // If the block is an out-of-date generation stamp or state,
                // but we're the standby, we shouldn't treat it as corrupt,
                // but instead just queue it for later processing.
                // Storing the reported block for later processing, as that is what
                // comes from the IBR / FBR and hence what we should use to compare
                // against the memory state.
                // See HDFS-6289 and HDFS-15422 for more context.
                queueReportedBlock(storageInfo, block, reportedState,
                    QUEUE_REASON_CORRUPT_STATE);
              } else {
                toCorrupt.add(c);
              }
              return storedBlock;
            } 
      1. DN1 sends IBR for b1gs2 and b1gs3 to SNN1.
      2. SNN1 processes b1sg2 and updates the blocks map.
      3. SNN1 queues b1gs3 for later because it determines that b1gs3 is a future genstamp.
      4. SNN1 loads b1gs3 edit logs and processes the queued reports for b1.
      5. SNN1 processes b1gs1 first and puts it back in the queue.
      6. SNN1 processes b1gs3 next and updates the blocks map.
      7. Later, SNN1 becomes the Active NameNode (ANN) during an HA Failover.
      8. SNN1 will catch to the latest edit logs, then process all queued block reports to become the ANN.
      9. ANN1 will process b1gs1 and mark it as corrupt.

      If the example above happens for every DN which stores b1, then when the HA failover happens, b1 will be incorrectly marked as corrupt. This will be fixed when the first DN sends a FullBlockReport or an IBR for b1.

      Logs from Active Cluster

      I added the following logs to confirm this issue in an active cluster:

      BlockToMarkCorrupt c = checkReplicaCorrupt(
          block, reportedState, storedBlock, ucState, dn);
      if (c != null) {
        DatanodeStorageInfo storedStorageInfo = storedBlock.findStorageInfo(dn);
        LOG.info("Found corrupt block {} [{}, {}] from DN {}. Stored block {} from DN {}",
            block, reportedState.name(), ucState.name(), storageInfo, storedBlock, storedStorageInfo);
        if (storageInfo.equals(storedStorageInfo) &&
              storedBlock.getGenerationStamp() > block.getGenerationStamp()) {
          LOG.info("Stored Block {} from the same DN {} has a newer GenStamp." +
              storedBlock, storedStorageInfo);
        }
        if (shouldPostponeBlocksFromFuture) {
          // If the block is an out-of-date generation stamp or state,
          // but we're the standby, we shouldn't treat it as corrupt,
          // but instead just queue it for later processing.
          // Storing the reported block for later processing, as that is what
          // comes from the IBR / FBR and hence what we should use to compare
          // against the memory state.
          // See HDFS-6289 and HDFS-15422 for more context.
          queueReportedBlock(storageInfo, block, reportedState,
              QUEUE_REASON_CORRUPT_STATE);
          LOG.info("Queueing the block {} for later processing", block);
        } else {
          toCorrupt.add(c);
          LOG.info("Marking the block {} as corrupt", block);
        }
        return storedBlock;
      } 

       

      Logs from nn1 (Active):

      2024-04-03T03:00:52.524-0700,INFO,[IPC Server handler 6 on default port 443],org.apache.hadoop.hdfs.server.namenode.FSNamesystem,"updatePipeline(blk_66092666802_65700910634, newGS=65700925027, newLength=10485760, newNodes=[[DN1]:10010, [DN2]:10010, [DN3]:10010, client=client1)"
      2024-04-03T03:00:52.539-0700,INFO,[IPC Server handler 6 on default port 443],org.apache.hadoop.hdfs.server.namenode.FSNamesystem,"updatePipeline(blk_66092666802_65700910634 => blk_66092666802_65700925027) success"
      2024-04-03T03:01:07.413-0700,INFO,[IPC Server handler 6 on default port 443],org.apache.hadoop.hdfs.server.namenode.FSNamesystem,"updatePipeline(blk_66092666802_65700925027, newGS=65700933553, newLength=20971520, newNodes=[[DN1]:10010, [DN2]:10010, [DN3]:10010, client=client1)"
      2024-04-03T03:01:07.413-0700,INFO,[IPC Server handler 6 on default port 443],org.apache.hadoop.hdfs.server.namenode.FSNamesystem,"updatePipeline(blk_66092666802_65700925027 => blk_66092666802_65700933553) success" 

       

      Logs from nn2 (Standby):

      2024-04-03T03:01:23.067-0700,INFO,[Block report processor],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Found corrupt block blk_66092666802_65700925027 [FINALIZED, COMPLETE] from DN [DISK]DS-1:NORMAL:[DN1]:10010. Stored block blk_66092666802_65700933553 from DN null"
      2024-04-03T03:01:23.067-0700,INFO,[Block report processor],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Queueing the block blk_66092666802_65700925027 for later processing"
      2024-04-03T03:01:24.159-0700,INFO,[Block report processor],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Found corrupt block blk_66092666802_65700925027 [FINALIZED, COMPLETE] from DN [DISK]DS-3:NORMAL:[DN3]:10010. Stored block blk_66092666802_65700933553 from DN null"
      2024-04-03T03:01:24.159-0700,INFO,[Block report processor],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Queueing the block blk_66092666802_65700925027 for later processing"
      2024-04-03T03:01:24.159-0700,INFO,[Block report processor],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Found corrupt block blk_66092666802_65700925027 [FINALIZED, COMPLETE] from DN [DISK]DS-2:NORMAL:[DN2]:10010. Stored block blk_66092666802_65700933553 from DN null"
      2024-04-03T03:01:24.159-0700,INFO,[Block report processor],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Queueing the block blk_66092666802_65700925027 for later processing" 

       

      Logs from nn2 when it transitions to Active:

      2024-04-03T15:39:09.050-0700,INFO,[IPC Server handler 8 on default port 8020],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Found corrupt block blk_66092666802_65700925027 [FINALIZED, COMPLETE] from DN [DISK]DS-1:NORMAL:[DN1]:10010. Stored block blk_66092666802_65700933553 from DN [DISK]DS-1:NORMAL:[DN1]:10010"
      2024-04-03T15:39:09.050-0700,INFO,[IPC Server handler 8 on default port 8020],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Stored Block blk_66092666802_65700933553 from the same DN [DISK]DS-1:NORMAL:[DN1]:10010 has a newer GenStamp."
      2024-04-03T15:39:09.050-0700,INFO,[IPC Server handler 8 on default port 8020],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Marking the block blk_66092666802_65700925027 as corrupt"
      2024-04-03T15:39:09.050-0700,INFO,[IPC Server handler 8 on default port 8020],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Found corrupt block blk_66092666802_65700925027 [FINALIZED, COMPLETE] from DN [DISK]DS-2:NORMAL:[DN2]:10010. Stored block blk_66092666802_65700933553 from DN [DISK]DS-2:NORMAL:[DN2]:10010"
      2024-04-03T15:39:09.050-0700,INFO,[IPC Server handler 8 on default port 8020],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Stored Block blk_66092666802_65700933553 from the same DN [DISK]DS-2:NORMAL:[DN2]:10010 has a newer GenStamp."
      2024-04-03T15:39:09.050-0700,INFO,[IPC Server handler 8 on default port 8020],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Marking the block blk_66092666802_65700925027 as corrupt"
      2024-04-03T15:39:09.050-0700,INFO,[IPC Server handler 8 on default port 8020],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Found corrupt block blk_66092666802_65700925027 [FINALIZED, COMPLETE] from DN [DISK]DS-3:NORMAL:[DN3]:10010. Stored block blk_66092666802_65700933553 from DN [DISK]DS-3:NORMAL:[DN3]:10010"
      2024-04-03T15:39:09.050-0700,INFO,[IPC Server handler 8 on default port 8020],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Stored Block blk_66092666802_65700933553 from the same DN [DISK]DS-3:NORMAL:[DN3]:10010 has a newer GenStamp."
      2024-04-03T15:39:09.050-0700,INFO,[IPC Server handler 8 on default port 8020],org.apache.hadoop.hdfs.server.blockmanagement.BlockManager,"Marking the block blk_66092666802_65700925027 as corrupt"
       

       

      Attachments

        Issue Links

          Activity

            People

              dannytbecker Danny Becker
              dannytbecker Danny Becker
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: