Browse Source

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

cnauroth 10 years ago
parent
commit
c637d6d9d5

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

@@ -671,6 +671,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

@@ -6581,6 +6581,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

@@ -170,6 +170,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);
   }
 
   /**