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 xg...@apache.org on 2017/07/31 16:01:06 UTC

[34/50] [abbrv] hadoop git commit: HDFS-12206. Rename the split EC / replicated block metrics.

HDFS-12206. Rename the split EC / replicated block metrics.


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

Branch: refs/heads/YARN-5734
Commit: 480c8db40c09cd0e25b4d145bc871b70a45d4f50
Parents: 77791e4
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jul 28 11:24:44 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jul 28 11:24:44 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  2 +-
 .../server/blockmanagement/BlockManager.java    | 22 ++---
 .../hdfs/server/namenode/FSNamesystem.java      | 92 ++++++++++----------
 .../namenode/metrics/ECBlockGroupsMBean.java    | 59 +++++++++++++
 .../metrics/ECBlockGroupsStatsMBean.java        | 59 -------------
 .../namenode/metrics/ReplicatedBlocksMBean.java | 63 ++++++++++++++
 .../metrics/ReplicatedBlocksStatsMBean.java     | 63 --------------
 .../server/namenode/TestAddStripedBlocks.java   |  4 +-
 .../server/namenode/TestNameNodeMXBean.java     |  8 +-
 .../namenode/TestReconstructStripedBlocks.java  |  4 +-
 .../namenode/metrics/TestNameNodeMetrics.java   | 22 ++---
 11 files changed, 199 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 1f60f32..d9568f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -1031,7 +1031,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_PIPELINE_ECN_ENABLED_DEFAULT = false;
 
   // Key Provider Cache Expiry
-  public static final String DFS_DATANODE_BLOCK_PINNING_ENABLED = 
+  public static final String DFS_DATANODE_BLOCK_PINNING_ENABLED =
     "dfs.datanode.block-pinning.enabled";
   public static final boolean DFS_DATANODE_BLOCK_PINNING_ENABLED_DEFAULT =
     false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index a5ee30b..fc754a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -232,47 +232,47 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /** Used by metrics. */
-  public long getLowRedundancyBlocksStat() {
+  public long getLowRedundancyBlocks() {
     return neededReconstruction.getLowRedundancyBlocksStat();
   }
 
   /** Used by metrics. */
-  public long getCorruptBlocksStat() {
+  public long getCorruptBlocks() {
     return corruptReplicas.getCorruptBlocksStat();
   }
 
   /** Used by metrics. */
-  public long getMissingBlocksStat() {
+  public long getMissingBlocks() {
     return neededReconstruction.getCorruptBlocksStat();
   }
 
   /** Used by metrics. */
-  public long getMissingReplicationOneBlocksStat() {
+  public long getMissingReplicationOneBlocks() {
     return neededReconstruction.getCorruptReplicationOneBlocksStat();
   }
 
   /** Used by metrics. */
-  public long getPendingDeletionBlocksStat() {
+  public long getPendingDeletionReplicatedBlocks() {
     return invalidateBlocks.getBlocksStat();
   }
 
   /** Used by metrics. */
-  public long getLowRedundancyECBlockGroupsStat() {
+  public long getLowRedundancyECBlockGroups() {
     return neededReconstruction.getLowRedundancyECBlockGroupsStat();
   }
 
   /** Used by metrics. */
-  public long getCorruptECBlockGroupsStat() {
+  public long getCorruptECBlockGroups() {
     return corruptReplicas.getCorruptECBlockGroupsStat();
   }
 
   /** Used by metrics. */
-  public long getMissingECBlockGroupsStat() {
+  public long getMissingECBlockGroups() {
     return neededReconstruction.getCorruptECBlockGroupsStat();
   }
 
   /** Used by metrics. */
-  public long getPendingDeletionECBlockGroupsStat() {
+  public long getPendingDeletionECBlockGroups() {
     return invalidateBlocks.getECBlockGroupsStat();
   }
 
@@ -2292,11 +2292,11 @@ public class BlockManager implements BlockStatsMXBean {
     return bmSafeMode.getBytesInFuture();
   }
 
-  public long getBytesInFutureReplicatedBlocksStat() {
+  public long getBytesInFutureReplicatedBlocks() {
     return bmSafeMode.getBytesInFutureBlocks();
   }
 
-  public long getBytesInFutureStripedBlocksStat() {
+  public long getBytesInFutureECBlockGroups() {
     return bmSafeMode.getBytesInFutureECBlockGroups();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/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 9872cd7..fd4ab8d 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
@@ -92,6 +92,7 @@ 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.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.server.namenode.metrics.ReplicatedBlocksMBean;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.monotonicNow;
@@ -243,10 +244,9 @@ import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
-import org.apache.hadoop.hdfs.server.namenode.metrics.ECBlockGroupsStatsMBean;
+import org.apache.hadoop.hdfs.server.namenode.metrics.ECBlockGroupsMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
-import org.apache.hadoop.hdfs.server.namenode.metrics.ReplicatedBlocksStatsMBean;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
@@ -340,7 +340,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 @InterfaceAudience.Private
 @Metrics(context="dfs")
 public class FSNamesystem implements Namesystem, FSNamesystemMBean,
-    NameNodeMXBean, ReplicatedBlocksStatsMBean, ECBlockGroupsStatsMBean {
+    NameNodeMXBean, ReplicatedBlocksMBean, ECBlockGroupsMBean {
   public static final Log LOG = LogFactory.getLog(FSNamesystem.class);
   private final MetricsRegistry registry = new MetricsRegistry("FSNamesystem");
   @Metric final MutableRatesWithAggregation detailedLockHoldTimeMetrics =
@@ -4076,10 +4076,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @see ClientProtocol#getBlocksStats()
    */
   BlocksStats getBlocksStats() {
-    return new BlocksStats(getLowRedundancyBlocksStat(),
-        getCorruptBlocksStat(), getMissingBlocksStat(),
-        getMissingReplicationOneBlocksStat(), getBlocksBytesInFutureStat(),
-        getPendingDeletionBlocksStat());
+    return new BlocksStats(getLowRedundancyReplicatedBlocks(),
+        getCorruptReplicatedBlocks(), getMissingReplicatedBlocks(),
+        getMissingReplicationOneBlocks(), getBytesInFutureReplicatedBlocks(),
+        getPendingDeletionReplicatedBlocks());
   }
 
   /**
@@ -4089,9 +4089,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @see ClientProtocol#getECBlockGroupsStats()
    */
   ECBlockGroupsStats getECBlockGroupsStats() {
-    return new ECBlockGroupsStats(getLowRedundancyECBlockGroupsStat(),
-        getCorruptECBlockGroupsStat(), getMissingECBlockGroupsStat(),
-        getECBlocksBytesInFutureStat(), getPendingDeletionECBlockGroupsStat());
+    return new ECBlockGroupsStats(getLowRedundancyECBlockGroups(),
+        getCorruptECBlockGroups(), getMissingECBlockGroups(),
+        getBytesInFutureECBlockGroups(), getPendingDeletionECBlockGroups());
   }
 
   @Override // FSNamesystemMBean
@@ -4638,76 +4638,76 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   @Override // ReplicatedBlocksMBean
   @Metric({"LowRedundancyReplicatedBlocks",
       "Number of low redundancy replicated blocks"})
-  public long getLowRedundancyBlocksStat() {
-    return blockManager.getLowRedundancyBlocksStat();
+  public long getLowRedundancyReplicatedBlocks() {
+    return blockManager.getLowRedundancyBlocks();
   }
 
   @Override // ReplicatedBlocksMBean
   @Metric({"CorruptReplicatedBlocks", "Number of corrupted replicated blocks"})
-  public long getCorruptBlocksStat() {
-    return blockManager.getCorruptBlocksStat();
+  public long getCorruptReplicatedBlocks() {
+    return blockManager.getCorruptBlocks();
   }
 
   @Override // ReplicatedBlocksMBean
   @Metric({"MissingReplicatedBlocks", "Number of missing replicated blocks"})
-  public long getMissingBlocksStat() {
-    return blockManager.getMissingBlocksStat();
+  public long getMissingReplicatedBlocks() {
+    return blockManager.getMissingBlocks();
   }
 
   @Override // ReplicatedBlocksMBean
-  @Metric({"MissingReplicatedOneBlocks", "Number of missing replicated blocks" +
-      " with replication factor 1"})
-  public long getMissingReplicationOneBlocksStat() {
-    return blockManager.getMissingReplicationOneBlocksStat();
+  @Metric({"MissingReplicationOneBlocks", "Number of missing replicated " +
+      "blocks with replication factor 1"})
+  public long getMissingReplicationOneBlocks() {
+    return blockManager.getMissingReplicationOneBlocks();
   }
 
   @Override // ReplicatedBlocksMBean
-  @Metric({"BytesReplicatedFutureBlocks", "Total bytes in replicated blocks " +
-      "with future generation stamp"})
-  public long getBlocksBytesInFutureStat() {
-    return blockManager.getBytesInFutureReplicatedBlocksStat();
+  @Metric({"BytesInFutureReplicatedBlocks", "Total bytes in replicated " +
+      "blocks with future generation stamp"})
+  public long getBytesInFutureReplicatedBlocks() {
+    return blockManager.getBytesInFutureReplicatedBlocks();
   }
 
   @Override // ReplicatedBlocksMBean
   @Metric({"PendingDeletionReplicatedBlocks", "Number of replicated blocks " +
       "that are pending deletion"})
-  public long getPendingDeletionBlocksStat() {
-    return blockManager.getPendingDeletionBlocksStat();
+  public long getPendingDeletionReplicatedBlocks() {
+    return blockManager.getPendingDeletionReplicatedBlocks();
   }
 
-  @Override // ECBlockGroupsStatsMBean
+  @Override // ECBlockGroupsMBean
   @Metric({"LowRedundancyECBlockGroups", "Number of erasure coded block " +
       "groups with low redundancy"})
-  public long getLowRedundancyECBlockGroupsStat() {
-    return blockManager.getLowRedundancyECBlockGroupsStat();
+  public long getLowRedundancyECBlockGroups() {
+    return blockManager.getLowRedundancyECBlockGroups();
   }
 
-  @Override // ECBlockGroupsStatsMBean
+  @Override // ECBlockGroupsMBean
   @Metric({"CorruptECBlockGroups", "Number of erasure coded block groups that" +
       " are corrupt"})
-  public long getCorruptECBlockGroupsStat() {
-    return blockManager.getCorruptECBlockGroupsStat();
+  public long getCorruptECBlockGroups() {
+    return blockManager.getCorruptECBlockGroups();
   }
 
-  @Override // ECBlockGroupsStatsMBean
+  @Override // ECBlockGroupsMBean
   @Metric({"MissingECBlockGroups", "Number of erasure coded block groups that" +
       " are missing"})
-  public long getMissingECBlockGroupsStat() {
-    return blockManager.getMissingECBlockGroupsStat();
+  public long getMissingECBlockGroups() {
+    return blockManager.getMissingECBlockGroups();
   }
 
-  @Override // ECBlockGroupsStatsMBean
-  @Metric({"BytesFutureECBlockGroups", "Total bytes in erasure coded block " +
+  @Override // ECBlockGroupsMBean
+  @Metric({"BytesInFutureECBlockGroups", "Total bytes in erasure coded block " +
       "groups with future generation stamp"})
-  public long getECBlocksBytesInFutureStat() {
-    return blockManager.getBytesInFutureStripedBlocksStat();
+  public long getBytesInFutureECBlockGroups() {
+    return blockManager.getBytesInFutureECBlockGroups();
   }
 
-  @Override // ECBlockGroupsStatsMBean
+  @Override // ECBlockGroupsMBean
   @Metric({"PendingDeletionECBlockGroups", "Number of erasure coded block " +
       "groups that are pending deletion"})
-  public long getPendingDeletionECBlockGroupsStat() {
-    return blockManager.getPendingDeletionECBlockGroupsStat();
+  public long getPendingDeletionECBlockGroups() {
+    return blockManager.getPendingDeletionECBlockGroups();
   }
 
   @Override
@@ -4774,9 +4774,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Register following MBeans with their respective names.
    * FSNamesystemMBean:
    *        "hadoop:service=NameNode,name=FSNamesystemState"
-   * ReplicatedBlocksStatsMBean:
+   * ReplicatedBlocksMBean:
    *        "hadoop:service=NameNode,name=ReplicatedBlocksState"
-   * ECBlockGroupsStatsMBean:
+   * ECBlockGroupsMBean:
    *        "hadoop:service=NameNode,name=ECBlockGroupsState"
    */
   private void registerMBean() {
@@ -4785,9 +4785,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       StandardMBean namesystemBean = new StandardMBean(
           this, FSNamesystemMBean.class);
       StandardMBean replicaBean = new StandardMBean(
-          this, ReplicatedBlocksStatsMBean.class);
+          this, ReplicatedBlocksMBean.class);
       StandardMBean ecBean = new StandardMBean(
-          this, ECBlockGroupsStatsMBean.class);
+          this, ECBlockGroupsMBean.class);
       namesystemMBeanName = MBeans.register(
           "NameNode", "FSNamesystemState", namesystemBean);
       replicatedBlocksMBeanName = MBeans.register(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsMBean.java
new file mode 100644
index 0000000..5fa646a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsMBean.java
@@ -0,0 +1,59 @@
+/**
+ * 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.server.namenode.metrics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * This interface defines the methods to get status pertaining to blocks of type
+ * {@link org.apache.hadoop.hdfs.protocol.BlockType#STRIPED} in FSNamesystem
+ * of a NameNode. It is also used for publishing via JMX.
+ * <p>
+ * Aggregated status of all blocks is reported in
+ * @see FSNamesystemMBean
+ * Name Node runtime activity statistic info is reported in
+ * @see org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics
+ *
+ */
+@InterfaceAudience.Private
+public interface ECBlockGroupsMBean {
+  /**
+   * Return count of erasure coded block groups with low redundancy.
+   */
+  long getLowRedundancyECBlockGroups();
+
+  /**
+   * Return count of erasure coded block groups that are corrupt.
+   */
+  long getCorruptECBlockGroups();
+
+  /**
+   * Return count of erasure coded block groups that are missing.
+   */
+  long getMissingECBlockGroups();
+
+  /**
+   * Return total bytes of erasure coded future block groups.
+   */
+  long getBytesInFutureECBlockGroups();
+
+  /**
+   * Return count of erasure coded block groups that are pending deletion.
+   */
+  long getPendingDeletionECBlockGroups();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsStatsMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsStatsMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsStatsMBean.java
deleted file mode 100644
index f9fd416..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsStatsMBean.java
+++ /dev/null
@@ -1,59 +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.server.namenode.metrics;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * This interface defines the methods to get status pertaining to blocks of type
- * {@link org.apache.hadoop.hdfs.protocol.BlockType#STRIPED} in FSNamesystem
- * of a NameNode. It is also used for publishing via JMX.
- * <p>
- * Aggregated status of all blocks is reported in
- * @see FSNamesystemMBean
- * Name Node runtime activity statistic info is reported in
- * @see org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics
- *
- */
-@InterfaceAudience.Private
-public interface ECBlockGroupsStatsMBean {
-  /**
-   * Return count of erasure coded block groups with low redundancy.
-   */
-  long getLowRedundancyECBlockGroupsStat();
-
-  /**
-   * Return count of erasure coded block groups that are corrupt.
-   */
-  long getCorruptECBlockGroupsStat();
-
-  /**
-   * Return count of erasure coded block groups that are missing.
-   */
-  long getMissingECBlockGroupsStat();
-
-  /**
-   * Return total bytes of erasure coded future block groups.
-   */
-  long getECBlocksBytesInFutureStat();
-
-  /**
-   * Return count of erasure coded block groups that are pending deletion.
-   */
-  long getPendingDeletionECBlockGroupsStat();
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ReplicatedBlocksMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ReplicatedBlocksMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ReplicatedBlocksMBean.java
new file mode 100644
index 0000000..e2c924e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ReplicatedBlocksMBean.java
@@ -0,0 +1,63 @@
+/**
+ * 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.server.namenode.metrics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * This interface defines the methods to get status pertaining to blocks of type
+ * {@link org.apache.hadoop.hdfs.protocol.BlockType#CONTIGUOUS} in FSNamesystem
+ * of a NameNode. It is also used for publishing via JMX.
+ * <p>
+ * Aggregated status of all blocks is reported in
+ * @see FSNamesystemMBean
+ * Name Node runtime activity statistic info is reported in
+ * @see org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics
+ */
+@InterfaceAudience.Private
+public interface ReplicatedBlocksMBean {
+  /**
+   * Return low redundancy blocks count.
+   */
+  long getLowRedundancyReplicatedBlocks();
+
+  /**
+   * Return corrupt blocks count.
+   */
+  long getCorruptReplicatedBlocks();
+
+  /**
+   * Return missing blocks count.
+   */
+  long getMissingReplicatedBlocks();
+
+  /**
+   * Return count of missing blocks with replication factor one.
+   */
+  long getMissingReplicationOneBlocks();
+
+  /**
+   * Return total bytes of future blocks.
+   */
+  long getBytesInFutureReplicatedBlocks();
+
+  /**
+   * Return count of blocks that are pending deletion.
+   */
+  long getPendingDeletionReplicatedBlocks();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ReplicatedBlocksStatsMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ReplicatedBlocksStatsMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ReplicatedBlocksStatsMBean.java
deleted file mode 100644
index 4643b80..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ReplicatedBlocksStatsMBean.java
+++ /dev/null
@@ -1,63 +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.server.namenode.metrics;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * This interface defines the methods to get status pertaining to blocks of type
- * {@link org.apache.hadoop.hdfs.protocol.BlockType#CONTIGUOUS} in FSNamesystem
- * of a NameNode. It is also used for publishing via JMX.
- * <p>
- * Aggregated status of all blocks is reported in
- * @see FSNamesystemMBean
- * Name Node runtime activity statistic info is reported in
- * @see org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics
- */
-@InterfaceAudience.Private
-public interface ReplicatedBlocksStatsMBean {
-  /**
-   * Return low redundancy blocks count.
-   */
-  long getLowRedundancyBlocksStat();
-
-  /**
-   * Return corrupt blocks count.
-   */
-  long getCorruptBlocksStat();
-
-  /**
-   * Return missing blocks count.
-   */
-  long getMissingBlocksStat();
-
-  /**
-   * Return count of missing blocks with replication factor one.
-   */
-  long getMissingReplicationOneBlocksStat();
-
-  /**
-   * Return total bytes of future blocks.
-   */
-  long getBlocksBytesInFutureStat();
-
-  /**
-   * Return count of blocks that are pending deletion.
-   */
-  long getPendingDeletionBlocksStat();
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index c556699..623c444 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -424,9 +424,9 @@ public class TestAddStripedBlocks {
         cluster.getDataNodes().get(3).getDatanodeId(), reports[0]);
     BlockManagerTestUtil.updateState(ns.getBlockManager());
     // the total number of corrupted block info is still 1
-    Assert.assertEquals(1, ns.getCorruptECBlockGroupsStat());
+    Assert.assertEquals(1, ns.getCorruptECBlockGroups());
     Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
-    Assert.assertEquals(0, ns.getCorruptBlocksStat());
+    Assert.assertEquals(0, ns.getCorruptReplicatedBlocks());
     // 2 internal blocks corrupted
     Assert.assertEquals(2, bm.getCorruptReplicas(stored).size());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index 32c2a49..63f9113 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -809,10 +809,10 @@ public class TestNameNodeMXBean {
       long totalMissingBlocks = cluster.getNamesystem().getMissingBlocksCount();
       Long replicaMissingBlocks =
           (Long) mbs.getAttribute(replStateMBeanName,
-              "MissingBlocksStat");
+              "MissingReplicatedBlocks");
       Long ecMissingBlocks =
           (Long) mbs.getAttribute(ecBlkGrpStateMBeanName,
-              "MissingECBlockGroupsStat");
+              "MissingECBlockGroups");
       assertEquals("Unexpected total missing blocks!",
           expectedMissingBlockCount, totalMissingBlocks);
       assertEquals("Unexpected total missing blocks!",
@@ -826,10 +826,10 @@ public class TestNameNodeMXBean {
           cluster.getNamesystem().getCorruptReplicaBlocks();
       Long replicaCorruptBlocks =
           (Long) mbs.getAttribute(replStateMBeanName,
-              "CorruptBlocksStat");
+              "CorruptReplicatedBlocks");
       Long ecCorruptBlocks =
           (Long) mbs.getAttribute(ecBlkGrpStateMBeanName,
-              "CorruptECBlockGroupsStat");
+              "CorruptECBlockGroups");
       assertEquals("Unexpected total corrupt blocks!",
           expectedCorruptBlockCount, totalCorruptBlocks);
       assertEquals("Unexpected total corrupt blocks!",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReconstructStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReconstructStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReconstructStripedBlocks.java
index 540ae63..02075f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReconstructStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReconstructStripedBlocks.java
@@ -419,7 +419,7 @@ public class TestReconstructStripedBlocks {
 
       // Verify low redundancy count matching EC block groups count
       BlockManagerTestUtil.updateState(bm);
-      assertEquals(blockGroups, bm.getLowRedundancyECBlockGroupsStat());
+      assertEquals(blockGroups, bm.getLowRedundancyECBlockGroups());
       DFSTestUtil.verifyClientStats(conf, dfsCluster);
 
 
@@ -429,7 +429,7 @@ public class TestReconstructStripedBlocks {
 
       // Verify pending reconstruction count
       assertEquals(blockGroups, getNumberOfBlocksToBeErasureCoded(dfsCluster));
-      assertEquals(0, bm.getLowRedundancyECBlockGroupsStat());
+      assertEquals(0, bm.getLowRedundancyECBlockGroups());
       DFSTestUtil.verifyClientStats(conf, dfsCluster);
     } finally {
       dfsCluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/480c8db4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index f598d8c..c3bb255 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -317,27 +317,27 @@ public class TestNameNodeMetrics {
         namesystem.getUnderReplicatedBlocks());
     assertEquals("Low redundancy metrics not matching!",
         namesystem.getLowRedundancyBlocks(),
-        namesystem.getLowRedundancyBlocksStat() +
-            namesystem.getLowRedundancyECBlockGroupsStat());
+        namesystem.getLowRedundancyReplicatedBlocks() +
+            namesystem.getLowRedundancyECBlockGroups());
     assertEquals("Corrupt blocks metrics not matching!",
         namesystem.getCorruptReplicaBlocks(),
-        namesystem.getCorruptBlocksStat() +
-            namesystem.getCorruptECBlockGroupsStat());
+        namesystem.getCorruptReplicatedBlocks() +
+            namesystem.getCorruptECBlockGroups());
     assertEquals("Missing blocks metrics not matching!",
         namesystem.getMissingBlocksCount(),
-        namesystem.getMissingBlocksStat() +
-            namesystem.getMissingECBlockGroupsStat());
+        namesystem.getMissingReplicatedBlocks() +
+            namesystem.getMissingECBlockGroups());
     assertEquals("Missing blocks with replication factor one not matching!",
         namesystem.getMissingReplOneBlocksCount(),
-        namesystem.getMissingReplicationOneBlocksStat());
+        namesystem.getMissingReplicationOneBlocks());
     assertEquals("Bytes in future blocks metrics not matching!",
         namesystem.getBytesInFuture(),
-        namesystem.getBlocksBytesInFutureStat() +
-            namesystem.getECBlocksBytesInFutureStat());
+        namesystem.getBytesInFutureReplicatedBlocks() +
+            namesystem.getBytesInFutureECBlockGroups());
     assertEquals("Pending deletion blocks metrics not matching!",
         namesystem.getPendingDeletionBlocks(),
-        namesystem.getPendingDeletionBlocksStat() +
-            namesystem.getPendingDeletionECBlockGroupsStat());
+        namesystem.getPendingDeletionReplicatedBlocks() +
+            namesystem.getPendingDeletionECBlockGroups());
   }
 
   /** Corrupt a block and ensure metrics reflects it */


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