Procházet zdrojové kódy

HDFS-9653. Added blocks pending deletion report to dfsadmin.
(Weiwei Yang via eyang)

Eric Yang před 9 roky
rodič
revize
10a2bc0dff

+ 24 - 10
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -1957,10 +1957,11 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  private long[] callGetStats() throws IOException {
+  private long getStateByIndex(int stateIndex) throws IOException {
     checkOpen();
     try (TraceScope ignored = tracer.newScope("getStats")) {
-      return namenode.getStats();
+      long[] states =  namenode.getStats();
+      return states.length > stateIndex ? states[stateIndex] : -1;
     }
   }
 
@@ -1968,8 +1969,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getStats()
    */
   public FsStatus getDiskStatus() throws IOException {
-    long rawNums[] = callGetStats();
-    return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
+    return new FsStatus(getStateByIndex(0),
+        getStateByIndex(1), getStateByIndex(2));
   }
 
   /**
@@ -1978,7 +1979,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getMissingBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_MISSING_BLOCKS_IDX);
   }
 
   /**
@@ -1987,8 +1989,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getMissingReplOneBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.
-        GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX);
+  }
+
+  /**
+   * Returns count of blocks pending on deletion.
+   * @throws IOException
+   */
+  public long getPendingDeletionBlocksCount() throws IOException {
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_PENDING_DELETION_BLOCKS_IDX);
   }
 
   /**
@@ -1996,7 +2007,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getUnderReplicatedBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_UNDER_REPLICATED_IDX);
   }
 
   /**
@@ -2004,7 +2016,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getCorruptBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_CORRUPT_BLOCKS_IDX);
   }
 
   /**
@@ -2014,7 +2027,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getBytesInFutureBlocks() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX);
   }
 
   /**

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -1114,6 +1114,15 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.getMissingBlocksCount();
   }
 
+  /**
+   * Returns count of blocks pending on deletion.
+   *
+   * @throws IOException
+   */
+  public long getPendingDeletionBlocksCount() throws IOException {
+    return dfs.getPendingDeletionBlocksCount();
+  }
+
   /**
    * Returns count of blocks with replication factor 1 and have
    * lost the only replica.

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -714,7 +714,8 @@ public interface ClientProtocol {
   int GET_STATS_MISSING_BLOCKS_IDX = 5;
   int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
   int GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX = 7;
-  int STATS_ARRAY_LENGTH = 8;
+  int GET_STATS_PENDING_DELETION_BLOCKS_IDX = 8;
+  int STATS_ARRAY_LENGTH = 9;
 
   /**
    * Get a set of statistics about the filesystem.
@@ -729,6 +730,7 @@ public interface ClientProtocol {
    * <li> [6] contains number of blocks which have replication factor
    *          1 and have lost the only replica. </li>
    * <li> [7] contains number of bytes  that are at risk for deletion. </li>
+   * <li> [8] contains number of pending deletion blocks. </li>
    * </ul>
    * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of
    * actual numbers to index into the array.

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -1562,6 +1562,8 @@ public class PBHelperClient {
         res.getMissingReplOneBlocks();
     result[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX] =
         res.hasBlocksInFuture() ? res.getBlocksInFuture() : 0;
+    result[ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX] =
+        res.getPendingDeletionBlocks();
     return result;
   }
 
@@ -1931,6 +1933,11 @@ public class PBHelperClient {
       result.setBlocksInFuture(
           fsStats[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX]);
     }
+    if (fsStats.length >=
+        ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX + 1) {
+      result.setPendingDeletionBlocks(
+          fsStats[ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX]);
+    }
     return result.build();
   }
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto

@@ -307,6 +307,7 @@ message GetFsStatsResponseProto {
   required uint64 missing_blocks = 6;
   optional uint64 missing_repl_one_blocks = 7;
   optional uint64 blocks_in_future = 8;
+  optional uint64 pending_deletion_blocks = 9;
 }
 
 enum DatanodeReportTypeProto {  // type of the datanode report

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

@@ -60,6 +60,9 @@ Trunk (Unreleased)
 
   IMPROVEMENTS
 
+    HDFS-9653.  Added blocks pending deletion report to dfsadmin.  
+    (Weiwei Yang via eyang)
+
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
     (Junping Du via llu)
 

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

@@ -184,6 +184,7 @@ class HeartbeatManager implements DatanodeStatistics {
                        -1L,
                        -1L,
                        -1L,
+                       -1L,
                        -1L};
   }
 

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

@@ -3809,6 +3809,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         getMissingReplOneBlocksCount();
     stats[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX] =
         blockManager.getBytesInFuture();
+    stats[ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX] =
+        blockManager.getPendingDeletionBlocksCount();
     return stats;
   }
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -502,6 +502,8 @@ public class DFSAdmin extends FsShell {
                        dfs.getMissingBlocksCount());
     System.out.println("Missing blocks (with replication factor 1): " +
                       dfs.getMissingReplOneBlocksCount());
+    System.out.println("Pending deletion blocks: " +
+        dfs.getPendingDeletionBlocksCount());
 
     System.out.println();
 

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

@@ -17,16 +17,19 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import java.lang.reflect.Method;
 import java.text.SimpleDateFormat;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
@@ -88,15 +91,31 @@ public class TestPendingInvalidateBlock {
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(REPLICATION, cluster.getNamesystem()
         .getPendingDeletionBlocks());
+    Assert.assertEquals(REPLICATION,
+        dfs.getPendingDeletionBlocksCount());
     Thread.sleep(6000);
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
+    Assert.assertEquals(0, dfs.getPendingDeletionBlocksCount());
     long nnStarted = cluster.getNamesystem().getNNStartedTimeInMillis();
     long blockDeletionStartTime = cluster.getNamesystem()
         .getBlockDeletionStartTime();
     Assert.assertTrue(String.format(
         "Expect blockDeletionStartTime = %d > nnStarted = %d.",
         blockDeletionStartTime, nnStarted), blockDeletionStartTime > nnStarted);
+
+    // test client protocol compatibility
+    Method method = DFSClient.class.
+        getDeclaredMethod("getStateByIndex", int.class);
+    method.setAccessible(true);
+    // get number of pending deletion blocks by its index
+    long validState = (Long) method.invoke(dfs.getClient(),
+        ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX);
+    // get an out of index value
+    long invalidState = (Long) method.invoke(dfs.getClient(),
+        ClientProtocol.STATS_ARRAY_LENGTH);
+    Assert.assertEquals(0, validState);
+    Assert.assertEquals(-1, invalidState);
   }
 
   /**