Bläddra i källkod

HDFS-6385. Show when block deletion will start after NameNode startup in WebUI. Contributed by Chris Nauroth.

(cherry picked from commit c637d6d9d5cb36b8cd5180787c5d25b68f2aa370)
(cherry picked from commit caded2545c3c773cf6e9cd21e49dcb6858a4de04)
cnauroth 10 år sedan
förälder
incheckning
3283268d8b

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

@@ -294,6 +294,9 @@ Release 2.6.0 - UNRELEASED
 
     HDFS-7230. Add rolling downgrade documentation. (szetszwo via jing9)
 
+    HDFS-6385. Show when block deletion will start after NameNode startup in
+    WebUI. (cnauroth)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)

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

@@ -121,6 +121,7 @@ public class BlockManager {
   private volatile long scheduledReplicationBlocksCount = 0L;
   private final AtomicLong excessBlocksCount = new AtomicLong(0L);
   private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
+  private final long startupDelayBlockDeletionInMs;
   
   /** Used by metrics */
   public long getPendingReplicationBlocksCount() {
@@ -143,6 +144,10 @@ public class BlockManager {
     return invalidateBlocks.numBlocks();
   }
   /** Used by metrics */
+  public long getStartupDelayBlockDeletionInMs() {
+    return startupDelayBlockDeletionInMs;
+  }
+  /** Used by metrics */
   public long getExcessBlocksCount() {
     return excessBlocksCount.get();
   }
@@ -266,11 +271,11 @@ public class BlockManager {
     datanodeManager = new DatanodeManager(this, namesystem, conf);
     heartbeatManager = datanodeManager.getHeartbeatManager();
 
-    final long pendingPeriod = conf.getLong(
+    startupDelayBlockDeletionInMs = conf.getLong(
         DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_KEY,
         DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_DEFAULT) * 1000L;
     invalidateBlocks = new InvalidateBlocks(
-        datanodeManager.blockInvalidateLimit, pendingPeriod);
+        datanodeManager.blockInvalidateLimit, startupDelayBlockDeletionInMs);
 
     // Compute the map capacity by allocating 2% of total memory
     blocksMap = new BlocksMap(

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -6575,6 +6575,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return blockManager.getPendingDeletionBlocksCount();
   }
 
+  @Override
+  public long getBlockDeletionStartTime() {
+    return startTime + blockManager.getStartupDelayBlockDeletionInMs();
+  }
+
   @Metric
   public long getExcessBlocks() {
     return blockManager.getExcessBlocksCount();

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java

@@ -152,6 +152,12 @@ public interface FSNamesystemMBean {
    */
   long getPendingDeletionBlocks();
 
+  /**
+   * Time when block deletions will begin
+   * @return time when block deletions will begin
+   */
+  long getBlockDeletionStartTime();
+
   /**
    * Number of content stale storages.
    * @return number of content stale storages

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -166,6 +166,7 @@
   <tr><th><a href="#tab-datanode">Decommissioning Nodes</a></th><td>{NumDecommissioningDataNodes}</td></tr>
   <tr><th title="Excludes missing blocks.">Number of Under-Replicated Blocks</th><td>{UnderReplicatedBlocks}</td></tr>
   <tr><th>Number of Blocks Pending Deletion</th><td>{PendingDeletionBlocks}</td></tr>
+  <tr><th>Block Deletion Start Time</th><td>{BlockDeletionStartTime|date_tostring}</td></tr>
 {/fs}
 </table>
 

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import java.text.SimpleDateFormat;
+
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -88,6 +90,15 @@ public class TestPendingInvalidateBlock {
     Thread.sleep(6000);
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
+    String nnStartedStr = cluster.getNamesystem().getNNStarted();
+    long nnStarted = new SimpleDateFormat("EEE MMM dd HH:mm:ss zzz yyyy")
+        .parse(nnStartedStr).getTime();
+    long blockDeletionStartTime = cluster.getNamesystem()
+        .getBlockDeletionStartTime();
+    Assert.assertTrue(String.format(
+        "Expect blockDeletionStartTime = %d > nnStarted = %d/nnStartedStr = %s.",
+        blockDeletionStartTime, nnStarted, nnStartedStr),
+        blockDeletionStartTime > nnStarted);
   }
 
   /**