Browse Source

HDFS-5004. Add additional JMX bean for NameNode status data. Contributed by Trevor Lorimer.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1507530 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Boudnik 12 years ago
parent
commit
dc17bda4b6

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

@@ -126,6 +126,9 @@ Trunk (Unreleased)
 
     HDFS-4904. Remove JournalService. (Arpit Agarwal via cnauroth)
 
+    HDFS-5004. Add additional JMX bean for NameNode status data
+    (Trevor Lorimer via cos)
+
   OPTIMIZATIONS
 
   BUG FIXES

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

@@ -6346,6 +6346,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return dir;
   }
 
+  @Override  // NameNodeMXBean
+  public String getCorruptFiles() {
+    List<String> list = new ArrayList<String>();
+    Collection<FSNamesystem.CorruptFileBlockInfo> corruptFileBlocks;
+    try {
+      corruptFileBlocks = listCorruptFileBlocks("/", null);
+      int corruptFileCount = corruptFileBlocks.size();
+      if (corruptFileCount != 0) {
+        for (FSNamesystem.CorruptFileBlockInfo c : corruptFileBlocks) {
+          list.add(c.toString());
+        }
+      }
+    } catch (IOException e) {
+      LOG.warn("Get corrupt file blocks returned error: " + e.getMessage());
+    }
+    return JSON.toString(list);
+  }
+
   /**
    * Verifies that the given identifier and password are valid and match.
    * @param identifier Token identifier.

+ 40 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -67,6 +67,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
@@ -123,7 +124,7 @@ import com.google.common.collect.Lists;
  * NameNode state, for example partial blocksMap etc.
  **********************************************************/
 @InterfaceAudience.Private
-public class NameNode {
+public class NameNode implements NameNodeStatusMXBean {
   static{
     HdfsConfiguration.init();
   }
@@ -531,6 +532,7 @@ public class NameNode {
   /** Start the services common to active and standby states */
   private void startCommonServices(Configuration conf) throws IOException {
     namesystem.startCommonServices(conf, haContext);
+    registerNNSMXBean();
     if (NamenodeRole.NAMENODE != role) {
       startHttpServer(conf);
       httpServer.setNameNodeAddress(getNameNodeAddress());
@@ -1368,6 +1370,43 @@ public class NameNode {
     return state.getServiceState();
   }
 
+  /**
+   * Register NameNodeStatusMXBean
+   */
+  private void registerNNSMXBean() {
+    MBeans.register("NameNode", "NameNodeStatus", this);
+  }
+
+  @Override // NameNodeStatusMXBean
+  public String getNNRole() {
+    String roleStr = "";
+    NamenodeRole role = getRole();
+    if (null != role) {
+      roleStr = role.toString();
+    }
+    return roleStr;
+  }
+
+  @Override // NameNodeStatusMXBean
+  public String getState() {
+    String servStateStr = "";
+    HAServiceState servState = getServiceState();
+    if (null != servState) {
+      servStateStr = servState.toString();
+    }
+    return servStateStr;
+  }
+
+  @Override // NameNodeStatusMXBean
+  public String getHostAndPort() {
+    return getNameNodeAddressHostPortString();
+  }
+
+  @Override // NameNodeStatusMXBean
+  public boolean isSecurityEnabled() {
+    return UserGroupInformation.isSecurityEnabled();
+  }
+
   /**
    * Shutdown the NN immediately in an ungraceful way. Used when it would be
    * unsafe for the NN to continue operating, e.g. during a failed HA state

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java

@@ -202,4 +202,11 @@ public interface NameNodeMXBean {
    * @return the compilation information, as a JSON string.
    */
   public String getCompileInfo();
+
+  /**
+   * Get the list of corrupt files
+   *
+   * @return the list of corrupt files, as a JSON string.
+   */
+  public String getCorruptFiles();
 }

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeStatusMXBean.java

@@ -0,0 +1,57 @@
+/**
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This is the JMX management interface for NameNode status information
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface NameNodeStatusMXBean {
+
+  /**
+   * Gets the NameNode role.
+   *
+   * @return the NameNode role.
+   */
+  public String getNNRole();
+
+  /**
+   * Gets the NameNode state.
+   *
+   * @return the NameNode state.
+   */
+  public String getState();
+
+  /**
+   * Gets the host and port colon separated.
+   *
+   * @return host and port colon separated.
+   */
+  public String getHostAndPort();
+
+  /**
+   * Gets if security is enabled.
+   *
+   * @return true, if security is enabled.
+   */
+  public boolean isSecurityEnabled();
+}

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java

@@ -126,6 +126,10 @@ public class TestNameNodeMXBean {
       // get attribute "CompileInfo"
       String compileInfo = (String) mbs.getAttribute(mxbeanName, "CompileInfo");
       assertEquals("Bad value for CompileInfo", fsn.getCompileInfo(), compileInfo);
+      // get attribute CorruptFiles
+      String corruptFiles = (String) (mbs.getAttribute(mxbeanName,
+          "CorruptFiles"));
+      assertEquals("Bad value for CorruptFiles", fsn.getCorruptFiles(), corruptFiles);
       // get attribute NameDirStatuses
       String nameDirStatuses = (String) (mbs.getAttribute(mxbeanName,
           "NameDirStatuses"));