You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by in...@apache.org on 2017/09/08 20:57:37 UTC

[05/45] hadoop git commit: HDFS-12218. Rename split EC / replicated block metrics in BlockManager.

HDFS-12218. Rename split EC / replicated block metrics in BlockManager.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/40c2f31f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/40c2f31f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/40c2f31f

Branch: refs/heads/HDFS-10467
Commit: 40c2f31f8dd45bc94291535ad41ffe3cc30b5536
Parents: 52b894d
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Sep 7 16:56:35 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Sep 7 16:56:35 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/BlocksStats.java       | 90 --------------------
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  4 +-
 .../hadoop/hdfs/protocol/ECBlockGroupStats.java | 83 ++++++++++++++++++
 .../hdfs/protocol/ECBlockGroupsStats.java       | 83 ------------------
 .../hdfs/protocol/ReplicatedBlockStats.java     | 90 ++++++++++++++++++++
 .../ClientNamenodeProtocolTranslatorPB.java     |  8 +-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 38 ++++-----
 .../hdfs/server/namenode/FSNamesystem.java      | 13 +--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  8 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  | 26 +++---
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     | 30 +++----
 11 files changed, 237 insertions(+), 236 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlocksStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlocksStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlocksStats.java
deleted file mode 100644
index 7eb30ca..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlocksStats.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Get statistics pertaining to blocks of type {@link BlockType#CONTIGUOUS}
- * in the filesystem.
- * <p>
- * @see ClientProtocol#getBlocksStats()
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class BlocksStats {
-  private final long lowRedundancyBlocksStat;
-  private final long corruptBlocksStat;
-  private final long missingBlocksStat;
-  private final long missingReplicationOneBlocksStat;
-  private final long bytesInFutureBlocksStat;
-  private final long pendingDeletionBlocksStat;
-
-  public BlocksStats(long lowRedundancyBlocksStat,
-      long corruptBlocksStat, long missingBlocksStat,
-      long missingReplicationOneBlocksStat, long bytesInFutureBlocksStat,
-      long pendingDeletionBlocksStat) {
-    this.lowRedundancyBlocksStat = lowRedundancyBlocksStat;
-    this.corruptBlocksStat = corruptBlocksStat;
-    this.missingBlocksStat = missingBlocksStat;
-    this.missingReplicationOneBlocksStat = missingReplicationOneBlocksStat;
-    this.bytesInFutureBlocksStat = bytesInFutureBlocksStat;
-    this.pendingDeletionBlocksStat = pendingDeletionBlocksStat;
-  }
-
-  public long getLowRedundancyBlocksStat() {
-    return lowRedundancyBlocksStat;
-  }
-
-  public long getCorruptBlocksStat() {
-    return corruptBlocksStat;
-  }
-
-  public long getMissingReplicaBlocksStat() {
-    return missingBlocksStat;
-  }
-
-  public long getMissingReplicationOneBlocksStat() {
-    return missingReplicationOneBlocksStat;
-  }
-
-  public long getBytesInFutureBlocksStat() {
-    return bytesInFutureBlocksStat;
-  }
-
-  public long getPendingDeletionBlocksStat() {
-    return pendingDeletionBlocksStat;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder statsBuilder = new StringBuilder();
-    statsBuilder.append("ReplicatedBlocksStats=[")
-        .append("LowRedundancyBlocks=").append(getLowRedundancyBlocksStat())
-        .append(", CorruptBlocks=").append(getCorruptBlocksStat())
-        .append(", MissingReplicaBlocks=").append(getMissingReplicaBlocksStat())
-        .append(", MissingReplicationOneBlocks=").append(
-            getMissingReplicationOneBlocksStat())
-        .append(", BytesInFutureBlocks=").append(getBytesInFutureBlocksStat())
-        .append(", PendingDeletionBlocks=").append(
-            getPendingDeletionBlocksStat())
-        .append("]");
-    return statsBuilder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index b550467..6626e3b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -778,14 +778,14 @@ public interface ClientProtocol {
    * in the filesystem.
    */
   @Idempotent
-  BlocksStats getBlocksStats() throws IOException;
+  ReplicatedBlockStats getBlocksStats() throws IOException;
 
   /**
    * Get statistics pertaining to blocks of type {@link BlockType#STRIPED}
    * in the filesystem.
    */
   @Idempotent
-  ECBlockGroupsStats getECBlockGroupsStats() throws IOException;
+  ECBlockGroupStats getECBlockGroupsStats() throws IOException;
 
   /**
    * Get a report on the system's current datanodes.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
new file mode 100644
index 0000000..7258c43
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Get statistics pertaining to blocks of type {@link BlockType#STRIPED}
+ * in the filesystem.
+ * <p>
+ * @see ClientProtocol#getECBlockGroupsStats()
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ECBlockGroupStats {
+  private final long lowRedundancyBlockGroupsStat;
+  private final long corruptBlockGroupsStat;
+  private final long missingBlockGroupsStat;
+  private final long bytesInFutureBlockGroupsStat;
+  private final long pendingDeletionBlockGroupsStat;
+
+  public ECBlockGroupStats(long lowRedundancyBlockGroupsStat, long
+      corruptBlockGroupsStat, long missingBlockGroupsStat, long
+      bytesInFutureBlockGroupsStat, long pendingDeletionBlockGroupsStat) {
+    this.lowRedundancyBlockGroupsStat = lowRedundancyBlockGroupsStat;
+    this.corruptBlockGroupsStat = corruptBlockGroupsStat;
+    this.missingBlockGroupsStat = missingBlockGroupsStat;
+    this.bytesInFutureBlockGroupsStat = bytesInFutureBlockGroupsStat;
+    this.pendingDeletionBlockGroupsStat = pendingDeletionBlockGroupsStat;
+  }
+
+  public long getBytesInFutureBlockGroupsStat() {
+    return bytesInFutureBlockGroupsStat;
+  }
+
+  public long getCorruptBlockGroupsStat() {
+    return corruptBlockGroupsStat;
+  }
+
+  public long getLowRedundancyBlockGroupsStat() {
+    return lowRedundancyBlockGroupsStat;
+  }
+
+  public long getMissingBlockGroupsStat() {
+    return missingBlockGroupsStat;
+  }
+
+  public long getPendingDeletionBlockGroupsStat() {
+    return pendingDeletionBlockGroupsStat;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder statsBuilder = new StringBuilder();
+    statsBuilder.append("ECBlockGroupStats=[")
+        .append("LowRedundancyBlockGroups=").append(
+            getLowRedundancyBlockGroupsStat())
+        .append(", CorruptBlockGroups=").append(getCorruptBlockGroupsStat())
+        .append(", MissingBlockGroups=").append(getMissingBlockGroupsStat())
+        .append(", BytesInFutureBlockGroups=").append(
+            getBytesInFutureBlockGroupsStat())
+        .append(", PendingDeletionBlockGroups=").append(
+            getPendingDeletionBlockGroupsStat())
+        .append("]");
+    return statsBuilder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupsStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupsStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupsStats.java
deleted file mode 100644
index 80cf262..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupsStats.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Get statistics pertaining to blocks of type {@link BlockType#STRIPED}
- * in the filesystem.
- * <p>
- * @see ClientProtocol#getECBlockGroupsStats()
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class ECBlockGroupsStats {
-  private final long lowRedundancyBlockGroupsStat;
-  private final long corruptBlockGroupsStat;
-  private final long missingBlockGroupsStat;
-  private final long bytesInFutureBlockGroupsStat;
-  private final long pendingDeletionBlockGroupsStat;
-
-  public ECBlockGroupsStats(long lowRedundancyBlockGroupsStat, long
-      corruptBlockGroupsStat, long missingBlockGroupsStat, long
-      bytesInFutureBlockGroupsStat, long pendingDeletionBlockGroupsStat) {
-    this.lowRedundancyBlockGroupsStat = lowRedundancyBlockGroupsStat;
-    this.corruptBlockGroupsStat = corruptBlockGroupsStat;
-    this.missingBlockGroupsStat = missingBlockGroupsStat;
-    this.bytesInFutureBlockGroupsStat = bytesInFutureBlockGroupsStat;
-    this.pendingDeletionBlockGroupsStat = pendingDeletionBlockGroupsStat;
-  }
-
-  public long getBytesInFutureBlockGroupsStat() {
-    return bytesInFutureBlockGroupsStat;
-  }
-
-  public long getCorruptBlockGroupsStat() {
-    return corruptBlockGroupsStat;
-  }
-
-  public long getLowRedundancyBlockGroupsStat() {
-    return lowRedundancyBlockGroupsStat;
-  }
-
-  public long getMissingBlockGroupsStat() {
-    return missingBlockGroupsStat;
-  }
-
-  public long getPendingDeletionBlockGroupsStat() {
-    return pendingDeletionBlockGroupsStat;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder statsBuilder = new StringBuilder();
-    statsBuilder.append("ECBlockGroupsStats=[")
-        .append("LowRedundancyBlockGroups=").append(
-            getLowRedundancyBlockGroupsStat())
-        .append(", CorruptBlockGroups=").append(getCorruptBlockGroupsStat())
-        .append(", MissingBlockGroups=").append(getMissingBlockGroupsStat())
-        .append(", BytesInFutureBlockGroups=").append(
-            getBytesInFutureBlockGroupsStat())
-        .append(", PendingDeletionBlockGroups=").append(
-            getPendingDeletionBlockGroupsStat())
-        .append("]");
-    return statsBuilder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
new file mode 100644
index 0000000..c92dbc7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Get statistics pertaining to blocks of type {@link BlockType#CONTIGUOUS}
+ * in the filesystem.
+ * <p>
+ * @see ClientProtocol#getBlocksStats()
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ReplicatedBlockStats {
+  private final long lowRedundancyBlocksStat;
+  private final long corruptBlocksStat;
+  private final long missingBlocksStat;
+  private final long missingReplicationOneBlocksStat;
+  private final long bytesInFutureBlocksStat;
+  private final long pendingDeletionBlocksStat;
+
+  public ReplicatedBlockStats(long lowRedundancyBlocksStat,
+      long corruptBlocksStat, long missingBlocksStat,
+      long missingReplicationOneBlocksStat, long bytesInFutureBlocksStat,
+      long pendingDeletionBlocksStat) {
+    this.lowRedundancyBlocksStat = lowRedundancyBlocksStat;
+    this.corruptBlocksStat = corruptBlocksStat;
+    this.missingBlocksStat = missingBlocksStat;
+    this.missingReplicationOneBlocksStat = missingReplicationOneBlocksStat;
+    this.bytesInFutureBlocksStat = bytesInFutureBlocksStat;
+    this.pendingDeletionBlocksStat = pendingDeletionBlocksStat;
+  }
+
+  public long getLowRedundancyBlocksStat() {
+    return lowRedundancyBlocksStat;
+  }
+
+  public long getCorruptBlocksStat() {
+    return corruptBlocksStat;
+  }
+
+  public long getMissingReplicaBlocksStat() {
+    return missingBlocksStat;
+  }
+
+  public long getMissingReplicationOneBlocksStat() {
+    return missingReplicationOneBlocksStat;
+  }
+
+  public long getBytesInFutureBlocksStat() {
+    return bytesInFutureBlocksStat;
+  }
+
+  public long getPendingDeletionBlocksStat() {
+    return pendingDeletionBlocksStat;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder statsBuilder = new StringBuilder();
+    statsBuilder.append("ReplicatedBlocksStats=[")
+        .append("LowRedundancyBlocks=").append(getLowRedundancyBlocksStat())
+        .append(", CorruptBlocks=").append(getCorruptBlocksStat())
+        .append(", MissingReplicaBlocks=").append(getMissingReplicaBlocksStat())
+        .append(", MissingReplicationOneBlocks=").append(
+            getMissingReplicationOneBlocksStat())
+        .append(", BytesInFutureBlocks=").append(getBytesInFutureBlocksStat())
+        .append(", PendingDeletionBlocks=").append(
+            getPendingDeletionBlocksStat())
+        .append("]");
+    return statsBuilder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index ec7d93f..53d8804 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -61,7 +61,7 @@ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -73,7 +73,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -695,7 +695,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public BlocksStats getBlocksStats() throws IOException {
+  public ReplicatedBlockStats getBlocksStats() throws IOException {
     try {
       return PBHelperClient.convert(rpcProxy.getFsBlocksStats(null,
           VOID_GET_FS_REPLICABLOCKS_STATS_REQUEST));
@@ -705,7 +705,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ECBlockGroupsStats getECBlockGroupsStats() throws IOException {
+  public ECBlockGroupStats getECBlockGroupsStats() throws IOException {
     try {
       return PBHelperClient.convert(rpcProxy.getFsECBlockGroupsStats(null,
           VOID_GET_FS_ECBLOCKGROUPS_STATS_REQUEST));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 04f9686..684ad70 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -76,7 +76,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
@@ -92,7 +92,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
@@ -1810,17 +1810,17 @@ public class PBHelperClient {
     return result;
   }
 
-  public static BlocksStats convert(
+  public static ReplicatedBlockStats convert(
       GetFsBlocksStatsResponseProto res) {
-    return new BlocksStats(res.getLowRedundancy(),
+    return new ReplicatedBlockStats(res.getLowRedundancy(),
         res.getCorruptBlocks(), res.getMissingBlocks(),
         res.getMissingReplOneBlocks(), res.getBlocksInFuture(),
         res.getPendingDeletionBlocks());
   }
 
-  public static ECBlockGroupsStats convert(
+  public static ECBlockGroupStats convert(
       GetFsECBlockGroupsStatsResponseProto res) {
-    return new ECBlockGroupsStats(res.getLowRedundancy(),
+    return new ECBlockGroupStats(res.getLowRedundancy(),
         res.getCorruptBlocks(), res.getMissingBlocks(),
         res.getBlocksInFuture(), res.getPendingDeletionBlocks());
   }
@@ -2237,36 +2237,36 @@ public class PBHelperClient {
   }
 
   public static GetFsBlocksStatsResponseProto convert(
-      BlocksStats blocksStats) {
+      ReplicatedBlockStats replicatedBlockStats) {
     GetFsBlocksStatsResponseProto.Builder result =
         GetFsBlocksStatsResponseProto.newBuilder();
     result.setLowRedundancy(
-        blocksStats.getLowRedundancyBlocksStat());
+        replicatedBlockStats.getLowRedundancyBlocksStat());
     result.setCorruptBlocks(
-        blocksStats.getCorruptBlocksStat());
+        replicatedBlockStats.getCorruptBlocksStat());
     result.setMissingBlocks(
-        blocksStats.getMissingReplicaBlocksStat());
+        replicatedBlockStats.getMissingReplicaBlocksStat());
     result.setMissingReplOneBlocks(
-        blocksStats.getMissingReplicationOneBlocksStat());
+        replicatedBlockStats.getMissingReplicationOneBlocksStat());
     result.setBlocksInFuture(
-        blocksStats.getBytesInFutureBlocksStat());
+        replicatedBlockStats.getBytesInFutureBlocksStat());
     result.setPendingDeletionBlocks(
-        blocksStats.getPendingDeletionBlocksStat());
+        replicatedBlockStats.getPendingDeletionBlocksStat());
     return result.build();
   }
 
   public static GetFsECBlockGroupsStatsResponseProto convert(
-      ECBlockGroupsStats ecBlockGroupsStats) {
+      ECBlockGroupStats ecBlockGroupStats) {
     GetFsECBlockGroupsStatsResponseProto.Builder result =
         GetFsECBlockGroupsStatsResponseProto.newBuilder();
     result.setLowRedundancy(
-        ecBlockGroupsStats.getLowRedundancyBlockGroupsStat());
-    result.setCorruptBlocks(ecBlockGroupsStats.getCorruptBlockGroupsStat());
-    result.setMissingBlocks(ecBlockGroupsStats.getMissingBlockGroupsStat());
+        ecBlockGroupStats.getLowRedundancyBlockGroupsStat());
+    result.setCorruptBlocks(ecBlockGroupStats.getCorruptBlockGroupsStat());
+    result.setMissingBlocks(ecBlockGroupStats.getMissingBlockGroupsStat());
     result.setBlocksInFuture(
-        ecBlockGroupsStats.getBytesInFutureBlockGroupsStat());
+        ecBlockGroupStats.getBytesInFutureBlockGroupsStat());
     result.setPendingDeletionBlocks(
-        ecBlockGroupsStats.getPendingDeletionBlockGroupsStat());
+        ecBlockGroupStats.getPendingDeletionBlockGroupsStat());
     return result.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index dedb11e..aada5bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -89,8 +89,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.server.namenode.metrics.ReplicatedBlocksMBean;
@@ -4082,8 +4083,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * <p>
    * @see ClientProtocol#getBlocksStats()
    */
-  BlocksStats getBlocksStats() {
-    return new BlocksStats(getLowRedundancyReplicatedBlocks(),
+  ReplicatedBlockStats getBlocksStats() {
+    return new ReplicatedBlockStats(getLowRedundancyReplicatedBlocks(),
         getCorruptReplicatedBlocks(), getMissingReplicatedBlocks(),
         getMissingReplicationOneBlocks(), getBytesInFutureReplicatedBlocks(),
         getPendingDeletionReplicatedBlocks());
@@ -4095,8 +4096,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * <p>
    * @see ClientProtocol#getECBlockGroupsStats()
    */
-  ECBlockGroupsStats getECBlockGroupsStats() {
-    return new ECBlockGroupsStats(getLowRedundancyECBlockGroups(),
+  ECBlockGroupStats getECBlockGroupsStats() {
+    return new ECBlockGroupStats(getLowRedundancyECBlockGroups(),
         getCorruptECBlockGroups(), getMissingECBlockGroups(),
         getBytesInFutureECBlockGroups(), getPendingDeletionECBlockGroups());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 3fbb7bd..7b14226 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -98,7 +98,7 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -116,7 +116,7 @@ import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -1163,14 +1163,14 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public BlocksStats getBlocksStats() throws IOException {
+  public ReplicatedBlockStats getBlocksStats() throws IOException {
     checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getBlocksStats();
   }
 
   @Override // ClientProtocol
-  public ECBlockGroupsStats getECBlockGroupsStats() throws IOException {
+  public ECBlockGroupStats getECBlockGroupsStats() throws IOException {
     checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getECBlockGroupsStats();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 88aafe2..a2bb2c05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -66,13 +66,13 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeVolumeInfo;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
@@ -534,30 +534,30 @@ public class DFSAdmin extends FsShell {
      * minutes. Use "-metaSave" to list of all such blocks and accurate 
      * counts.
      */
-    BlocksStats blocksStats = dfs.getClient().getNamenode().getBlocksStats();
+    ReplicatedBlockStats replicatedBlockStats = dfs.getClient().getNamenode().getBlocksStats();
     System.out.println("Replicated Blocks:");
     System.out.println("\tUnder replicated blocks: " +
-        blocksStats.getLowRedundancyBlocksStat());
+        replicatedBlockStats.getLowRedundancyBlocksStat());
     System.out.println("\tBlocks with corrupt replicas: " +
-        blocksStats.getCorruptBlocksStat());
+        replicatedBlockStats.getCorruptBlocksStat());
     System.out.println("\tMissing blocks: " +
-        blocksStats.getMissingReplicaBlocksStat());
+        replicatedBlockStats.getMissingReplicaBlocksStat());
     System.out.println("\tMissing blocks (with replication factor 1): " +
-        blocksStats.getMissingReplicationOneBlocksStat());
+        replicatedBlockStats.getMissingReplicationOneBlocksStat());
     System.out.println("\tPending deletion blocks: " +
-        blocksStats.getPendingDeletionBlocksStat());
+        replicatedBlockStats.getPendingDeletionBlocksStat());
 
-    ECBlockGroupsStats ecBlockGroupsStats =
+    ECBlockGroupStats ecBlockGroupStats =
         dfs.getClient().getNamenode().getECBlockGroupsStats();
     System.out.println("Erasure Coded Block Groups: ");
     System.out.println("\tLow redundancy block groups: " +
-        ecBlockGroupsStats.getLowRedundancyBlockGroupsStat());
+        ecBlockGroupStats.getLowRedundancyBlockGroupsStat());
     System.out.println("\tBlock groups with corrupt internal blocks: " +
-        ecBlockGroupsStats.getCorruptBlockGroupsStat());
+        ecBlockGroupStats.getCorruptBlockGroupsStat());
     System.out.println("\tMissing block groups: " +
-        ecBlockGroupsStats.getMissingBlockGroupsStat());
+        ecBlockGroupStats.getMissingBlockGroupsStat());
     System.out.println("\tPending deletion block groups: " +
-        ecBlockGroupsStats.getPendingDeletionBlockGroupsStat());
+        ecBlockGroupStats.getPendingDeletionBlockGroupsStat());
 
     System.out.println();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index f3572ff..0926b44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -117,8 +117,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -1667,36 +1667,36 @@ public class DFSTestUtil {
         cluster.getFileSystem(0).getUri(),
         ClientProtocol.class).getProxy();
     long[] aggregatedStats = cluster.getNameNode().getRpcServer().getStats();
-    BlocksStats blocksStats =
+    ReplicatedBlockStats replicatedBlockStats =
         client.getBlocksStats();
-    ECBlockGroupsStats ecBlockGroupsStats = client.getECBlockGroupsStats();
+    ECBlockGroupStats ecBlockGroupStats = client.getECBlockGroupsStats();
 
     assertEquals("Under replicated stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_LOW_REDUNDANCY_IDX],
         aggregatedStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
     assertEquals("Low redundancy stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_LOW_REDUNDANCY_IDX],
-        blocksStats.getLowRedundancyBlocksStat() +
-            ecBlockGroupsStats.getLowRedundancyBlockGroupsStat());
+        replicatedBlockStats.getLowRedundancyBlocksStat() +
+            ecBlockGroupStats.getLowRedundancyBlockGroupsStat());
     assertEquals("Corrupt blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX],
-        blocksStats.getCorruptBlocksStat() +
-            ecBlockGroupsStats.getCorruptBlockGroupsStat());
+        replicatedBlockStats.getCorruptBlocksStat() +
+            ecBlockGroupStats.getCorruptBlockGroupsStat());
     assertEquals("Missing blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX],
-        blocksStats.getMissingReplicaBlocksStat() +
-            ecBlockGroupsStats.getMissingBlockGroupsStat());
+        replicatedBlockStats.getMissingReplicaBlocksStat() +
+            ecBlockGroupStats.getMissingBlockGroupsStat());
     assertEquals("Missing blocks with replication factor one not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX],
-        blocksStats.getMissingReplicationOneBlocksStat());
+        replicatedBlockStats.getMissingReplicationOneBlocksStat());
     assertEquals("Bytes in future blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX],
-        blocksStats.getBytesInFutureBlocksStat() +
-            ecBlockGroupsStats.getBytesInFutureBlockGroupsStat());
+        replicatedBlockStats.getBytesInFutureBlocksStat() +
+            ecBlockGroupStats.getBytesInFutureBlockGroupsStat());
     assertEquals("Pending deletion blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX],
-        blocksStats.getPendingDeletionBlocksStat() +
-            ecBlockGroupsStats.getPendingDeletionBlockGroupsStat());
+        replicatedBlockStats.getPendingDeletionBlocksStat() +
+            ecBlockGroupStats.getPendingDeletionBlockGroupsStat());
   }
 
   /**


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org