You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2014/12/10 06:01:47 UTC

hbase git commit: HBASE-12665 When aborting, dump metrics Add some fixup of checkstyle fails

Repository: hbase
Updated Branches:
  refs/heads/master 9b808155e -> df2227075


    HBASE-12665 When aborting, dump metrics
    Add some fixup of checkstyle fails


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

Branch: refs/heads/master
Commit: df22270756be8556085a8f1f3910a2f0d7ebc933
Parents: 9b80815
Author: stack <st...@apache.org>
Authored: Tue Dec 9 21:01:15 2014 -0800
Committer: stack <st...@apache.org>
Committed: Tue Dec 9 21:01:15 2014 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HColumnDescriptor.java  |   3 +-
 .../java/org/apache/hadoop/hbase/AuthUtil.java  |   5 +
 .../java/org/apache/hadoop/hbase/CellUtil.java  |   6 +-
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java   | 286 +-------------
 .../hbase/regionserver/HRegionServer.java       |   9 +
 .../org/apache/hadoop/hbase/util/JSONBean.java  | 387 +++++++++++++++++++
 6 files changed, 425 insertions(+), 271 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/df222707/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
index 2f6d11e..5335bef 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
@@ -85,7 +85,8 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
   /**
    * Key for cache data into L1 if cache is set up with more than one tier.
    * To set in the shell, do something like this:
-   * <code>hbase(main):003:0> create 't', {NAME => 't', CONFIGURATION => {CACHE_DATA_IN_L1 => 'true'}}</code>
+   * <code>hbase(main):003:0> create 't',
+   *    {NAME => 't', CONFIGURATION => {CACHE_DATA_IN_L1 => 'true'}}</code>
    */
   public static final String CACHE_DATA_IN_L1 = "CACHE_DATA_IN_L1";
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/df222707/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
index bdc6837..282b5e3 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
@@ -39,6 +39,11 @@ import org.apache.hadoop.security.UserGroupInformation;
 @InterfaceStability.Evolving
 public class AuthUtil {
   private static final Log LOG = LogFactory.getLog(AuthUtil.class);
+
+  private AuthUtil() {
+    super();
+  }
+
   /**
    * Checks if security is enabled and if so, launches chore for refreshing kerberos ticket.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/df222707/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 1394f84..fefe626 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -171,7 +171,7 @@ public final class CellUtil {
     return new KeyValue(row, family, qualifier, timestamp, KeyValue.Type.codeToType(type), value);
   }
 
-  public static Cell createCell (final byte [] rowArray, final int rowOffset, final int rowLength,
+  public static Cell createCell(final byte [] rowArray, final int rowOffset, final int rowLength,
       final byte [] familyArray, final int familyOffset, final int familyLength,
       final byte [] qualifierArray, final int qualifierOffset, final int qualifierLength) {
     // See createCell(final byte [] row, final byte [] value) for why we default Maximum type.
@@ -567,7 +567,7 @@ public final class CellUtil {
   /********************* tags *************************************/
   /**
    * Util method to iterate through the tags
-   * 
+   *
    * @param tags
    * @param offset
    * @param length
@@ -883,4 +883,4 @@ public final class CellUtil {
 
     return builder.toString();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/df222707/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
index 74fcd86..498e213 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
@@ -20,24 +20,11 @@ package org.apache.hadoop.hbase.http.jmx;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.lang.management.ManagementFactory;
-import java.lang.reflect.Array;
-import java.util.Iterator;
-import java.util.Set;
 
-import javax.management.AttributeNotFoundException;
-import javax.management.InstanceNotFoundException;
-import javax.management.IntrospectionException;
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanException;
-import javax.management.MBeanInfo;
 import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
-import javax.management.ReflectionException;
-import javax.management.RuntimeMBeanException;
-import javax.management.RuntimeErrorException;
 import javax.management.openmbean.CompositeData;
-import javax.management.openmbean.CompositeType;
 import javax.management.openmbean.TabularData;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
@@ -47,8 +34,7 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.http.HttpServer;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.JsonGenerator;
+import org.apache.hadoop.hbase.util.JSONBean;
 
 /*
  * This servlet is based off of the JMXProxyServlet from Tomcat 7.0.14. It has
@@ -135,10 +121,8 @@ public class JMXJsonServlet extends HttpServlet {
    */
   protected transient MBeanServer mBeanServer;
 
-  /**
-   * Json Factory to create Json generators for write objects in json format
-   */
-  protected transient JsonFactory jsonFactory;
+  protected transient JSONBean jsonBeanWriter;
+
   /**
    * Initialize this servlet.
    */
@@ -146,7 +130,7 @@ public class JMXJsonServlet extends HttpServlet {
   public void init() throws ServletException {
     // Retrieve the MBean server
     mBeanServer = ManagementFactory.getPlatformMBeanServer();
-    jsonFactory = new JsonFactory();
+    this.jsonBeanWriter = new JSONBean();
   }
 
   /**
@@ -163,11 +147,12 @@ public class JMXJsonServlet extends HttpServlet {
       if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), request, response)) {
         return;
       }
-      JsonGenerator jg = null;
       String jsonpcb = null;
       PrintWriter writer = null;
+      JSONBean.Writer beanWriter = null;
       try {
         writer = response.getWriter();
+        beanWriter = this.jsonBeanWriter.open(writer);
  
         // "callback" parameter implies JSONP outpout
         jsonpcb = request.getParameter(CALLBACK_PARAM);
@@ -181,23 +166,22 @@ public class JMXJsonServlet extends HttpServlet {
         String tmpStr = request.getParameter(INCLUDE_DESCRIPTION);
         boolean description = tmpStr != null && tmpStr.length() > 0;
 
-        jg = jsonFactory.createJsonGenerator(writer);
-        jg.disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET);
-        jg.useDefaultPrettyPrinter();
-        jg.writeStartObject();
-
         // query per mbean attribute
         String getmethod = request.getParameter("get");
         if (getmethod != null) {
           String[] splitStrings = getmethod.split("\\:\\:");
           if (splitStrings.length != 2) {
-            jg.writeStringField("result", "ERROR");
-            jg.writeStringField("message", "query format is not as expected.");
-            jg.flush();
+            beanWriter.write("result", "ERROR");
+            beanWriter.write("message", "query format is not as expected.");
+            beanWriter.flush();
             response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
             return;
           }
-          listBeans(jg, new ObjectName(splitStrings[0]), splitStrings[1], description, response);
+          if (beanWriter.write(this.mBeanServer, new ObjectName(splitStrings[0]),
+              splitStrings[1], description) != 0) {
+            beanWriter.flush();
+            response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+          }
           return;
         }
 
@@ -206,11 +190,12 @@ public class JMXJsonServlet extends HttpServlet {
         if (qry == null) {
           qry = "*:*";
         }
-        listBeans(jg, new ObjectName(qry), null, description, response);
-      } finally {
-        if (jg != null) {
-          jg.close();
+        if (beanWriter.write(this.mBeanServer, new ObjectName(qry), null, description) != 0) {
+          beanWriter.flush();
+          response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
         }
+      } finally {
+        if (beanWriter != null) beanWriter.close();
         if (jsonpcb != null) {
            writer.write(");");
         }
@@ -226,237 +211,4 @@ public class JMXJsonServlet extends HttpServlet {
       response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
     }
   }
-
-  // --------------------------------------------------------- Private Methods
-  private void listBeans(JsonGenerator jg, ObjectName qry, String attribute,
-      final boolean description, final HttpServletResponse response) 
-  throws IOException {
-    LOG.trace("Listing beans for "+qry);
-    Set<ObjectName> names = null;
-    names = mBeanServer.queryNames(qry, null);
-    jg.writeArrayFieldStart("beans");
-    Iterator<ObjectName> it = names.iterator();
-    while (it.hasNext()) {
-      ObjectName oname = it.next();
-      MBeanInfo minfo;
-      String code = "";
-      String descriptionStr = null;
-      Object attributeinfo = null;
-      try {
-        minfo = mBeanServer.getMBeanInfo(oname);
-        code = minfo.getClassName();
-        if (description) descriptionStr = minfo.getDescription();
-        String prs = "";
-        try {
-          if ("org.apache.commons.modeler.BaseModelMBean".equals(code)) {
-            prs = "modelerType";
-            code = (String) mBeanServer.getAttribute(oname, prs);
-          }
-          if (attribute != null) {
-            prs = attribute;
-            attributeinfo = mBeanServer.getAttribute(oname, prs);
-          }
-        } catch (RuntimeMBeanException e) {
-         // UnsupportedOperationExceptions happen in the normal course of business,
-         // so no need to log them as errors all the time.
-         if (e.getCause() instanceof UnsupportedOperationException) {
-           if (LOG.isTraceEnabled()) {
-             LOG.trace("Getting attribute " + prs + " of " + oname + " threw " + e);
-           }
-         } else {
-           LOG.error("Getting attribute " + prs + " of " + oname + " threw an exception", e);
-         }
-         return;
-        } catch (AttributeNotFoundException e) {
-          // If the modelerType attribute was not found, the class name is used
-          // instead.
-          LOG.error("getting attribute " + prs + " of " + oname
-              + " threw an exception", e);
-        } catch (MBeanException e) {
-          // The code inside the attribute getter threw an exception so log it,
-          // and fall back on the class name
-          LOG.error("getting attribute " + prs + " of " + oname
-              + " threw an exception", e);
-        } catch (RuntimeException e) {
-          // For some reason even with an MBeanException available to them
-          // Runtime exceptionscan still find their way through, so treat them
-          // the same as MBeanException
-          LOG.error("getting attribute " + prs + " of " + oname
-              + " threw an exception", e);
-        } catch ( ReflectionException e ) {
-          // This happens when the code inside the JMX bean (setter?? from the
-          // java docs) threw an exception, so log it and fall back on the 
-          // class name
-          LOG.error("getting attribute " + prs + " of " + oname
-              + " threw an exception", e);
-        }
-      } catch (InstanceNotFoundException e) {
-        //Ignored for some reason the bean was not found so don't output it
-        continue;
-      } catch ( IntrospectionException e ) {
-        // This is an internal error, something odd happened with reflection so
-        // log it and don't output the bean.
-        LOG.error("Problem while trying to process JMX query: " + qry
-            + " with MBean " + oname, e);
-        continue;
-      } catch ( ReflectionException e ) {
-        // This happens when the code inside the JMX bean threw an exception, so
-        // log it and don't output the bean.
-        LOG.error("Problem while trying to process JMX query: " + qry
-            + " with MBean " + oname, e);
-        continue;
-      }
-
-      jg.writeStartObject();
-      jg.writeStringField("name", oname.toString());
-      if (description && descriptionStr != null && descriptionStr.length() > 0) {
-        jg.writeStringField("description", descriptionStr);
-      }
-      jg.writeStringField("modelerType", code);
-      if (attribute != null && attributeinfo == null) {
-        jg.writeStringField("result", "ERROR");
-        jg.writeStringField("message", "No attribute with name " + attribute + " was found.");
-        jg.writeEndObject();
-        jg.writeEndArray();
-        jg.close();
-        response.setStatus(HttpServletResponse.SC_NOT_FOUND);
-        return;
-      }
-      
-      if (attribute != null) {
-        writeAttribute(jg, attribute, descriptionStr, attributeinfo);
-      } else {
-        MBeanAttributeInfo attrs[] = minfo.getAttributes();
-        for (int i = 0; i < attrs.length; i++) {
-          writeAttribute(jg, oname, description, attrs[i]);
-        }
-      }
-      jg.writeEndObject();
-    }
-    jg.writeEndArray();
-  }
-
-  private void writeAttribute(JsonGenerator jg, ObjectName oname, final boolean description,
-      MBeanAttributeInfo attr)
-  throws IOException {
-    if (!attr.isReadable()) {
-      return;
-    }
-    String attName = attr.getName();
-    if ("modelerType".equals(attName)) {
-      return;
-    }
-    if (attName.indexOf("=") >= 0 || attName.indexOf(":") >= 0 || attName.indexOf(" ") >= 0) {
-      return;
-    }
-    String descriptionStr = description? attr.getDescription(): null;
-    Object value = null;
-    try {
-      value = mBeanServer.getAttribute(oname, attName);
-    } catch (RuntimeMBeanException e) {
-      // UnsupportedOperationExceptions happen in the normal course of business,
-      // so no need to log them as errors all the time.
-      if (e.getCause() instanceof UnsupportedOperationException) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Getting attribute " + attName + " of " + oname + " threw " + e);
-        }
-      } else {
-        LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
-      }
-      return;
-    } catch (RuntimeErrorException e) {
-      // RuntimeErrorException happens when an unexpected failure occurs in getAttribute
-      // for example https://issues.apache.org/jira/browse/DAEMON-120
-      LOG.debug("getting attribute "+attName+" of "+oname+" threw an exception", e);
-      return;
-    } catch (AttributeNotFoundException e) {
-      //Ignored the attribute was not found, which should never happen because the bean
-      //just told us that it has this attribute, but if this happens just don't output
-      //the attribute.
-      return;
-    } catch (MBeanException e) {
-      //The code inside the attribute getter threw an exception so log it, and
-      // skip outputting the attribute
-      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
-      return;
-    } catch (RuntimeException e) {
-      //For some reason even with an MBeanException available to them Runtime exceptions
-      //can still find their way through, so treat them the same as MBeanException
-      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
-      return;
-    } catch (ReflectionException e) {
-      //This happens when the code inside the JMX bean (setter?? from the java docs)
-      //threw an exception, so log it and skip outputting the attribute
-      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
-      return;
-    } catch (InstanceNotFoundException e) {
-      //Ignored the mbean itself was not found, which should never happen because we
-      //just accessed it (perhaps something unregistered in-between) but if this
-      //happens just don't output the attribute.
-      return;
-    }
-
-    writeAttribute(jg, attName, descriptionStr, value);
-  }
-  
-  private void writeAttribute(JsonGenerator jg, String attName, final String descriptionStr,
-      Object value)
-  throws IOException {
-    boolean description = false;
-    if (descriptionStr != null && descriptionStr.length() > 0 && !attName.equals(descriptionStr)) {
-      description = true;
-      jg.writeFieldName(attName);
-      jg.writeStartObject();
-      jg.writeFieldName("description");
-      jg.writeString(descriptionStr);
-      jg.writeFieldName("value");
-      writeObject(jg, description, value);
-      jg.writeEndObject();
-    } else {
-      jg.writeFieldName(attName);
-      writeObject(jg, description, value);
-    }
-  }
-  
-  private void writeObject(JsonGenerator jg, final boolean description, Object value)
-  throws IOException {
-    if(value == null) {
-      jg.writeNull();
-    } else {
-      Class<?> c = value.getClass();
-      if (c.isArray()) {
-        jg.writeStartArray();
-        int len = Array.getLength(value);
-        for (int j = 0; j < len; j++) {
-          Object item = Array.get(value, j);
-          writeObject(jg, description, item);
-        }
-        jg.writeEndArray();
-      } else if(value instanceof Number) {
-        Number n = (Number)value;
-        jg.writeNumber(n.toString());
-      } else if(value instanceof Boolean) {
-        Boolean b = (Boolean)value;
-        jg.writeBoolean(b);
-      } else if(value instanceof CompositeData) {
-        CompositeData cds = (CompositeData)value;
-        CompositeType comp = cds.getCompositeType();
-        Set<String> keys = comp.keySet();
-        jg.writeStartObject();
-        for (String key: keys) {
-          writeAttribute(jg, key, null, cds.get(key));
-        }
-        jg.writeEndObject();
-      } else if(value instanceof TabularData) {
-        TabularData tds = (TabularData)value;
-        jg.writeStartArray();
-        for(Object entry : tds.values()) {
-          writeObject(jg, description, entry);
-        }
-        jg.writeEndArray();
-      } else {
-        jg.writeString(value.toString());
-      }
-    }
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/df222707/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 3c7b769..ae749b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -48,6 +48,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.net.InetAddress;
 
+import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 import javax.servlet.http.HttpServlet;
 
@@ -140,6 +141,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HasThread;
+import org.apache.hadoop.hbase.util.JSONBean;
 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
 import org.apache.hadoop.hbase.util.Sleeper;
 import org.apache.hadoop.hbase.util.Threads;
@@ -1924,6 +1926,13 @@ public class HRegionServer extends HasThread implements
     // java.util.HashSet's toString() method to print the coprocessor names.
     LOG.fatal("RegionServer abort: loaded coprocessors are: " +
         CoprocessorHost.getLoadedCoprocessors());
+    // Try and dump metrics if abort -- might give clue as to how fatal came about....
+    try {
+      LOG.info("Dump of metrics as JSON on abort: " + JSONBean.dumpRegionServerMetrics());
+    } catch (MalformedObjectNameException | IOException e) {
+      LOG.warn("Failed dumping metrics", e);
+    }
+
     // Do our best to report our abort to the master, but this may not work
     try {
       if (cause != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/df222707/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
new file mode 100644
index 0000000..0739e91
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
@@ -0,0 +1,387 @@
+/*
+ * 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.hbase.util;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Array;
+import java.util.Iterator;
+import java.util.Set;
+
+import javax.management.AttributeNotFoundException;
+import javax.management.InstanceNotFoundException;
+import javax.management.IntrospectionException;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanException;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import javax.management.RuntimeErrorException;
+import javax.management.RuntimeMBeanException;
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.CompositeType;
+import javax.management.openmbean.TabularData;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonGenerator;
+
+/**
+ * Utility for doing JSON and MBeans.
+ */
+public class JSONBean {
+  private static final Log LOG = LogFactory.getLog(JSONBean.class);
+  private final JsonFactory jsonFactory;
+
+  public JSONBean() {
+    this.jsonFactory = new JsonFactory();
+  }
+
+  /**
+   * Use dumping out mbeans as JSON.
+   */
+  public interface Writer extends Closeable {
+    void write(final String key, final String value) throws JsonGenerationException, IOException;
+    int write(final MBeanServer mBeanServer, ObjectName qry, String attribute,
+        final boolean description) throws IOException;
+    void flush() throws IOException;
+  }
+
+  public Writer open(final PrintWriter writer) throws IOException {
+    final JsonGenerator jg = jsonFactory.createJsonGenerator(writer);
+    jg.disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET);
+    jg.useDefaultPrettyPrinter();
+    jg.writeStartObject();
+    return new Writer() {
+      @Override
+      public void flush() throws IOException {
+        jg.flush();
+      }
+
+      @Override
+      public void close() throws IOException {
+        jg.close();
+      }
+
+      @Override
+      public void write(String key, String value) throws JsonGenerationException, IOException {
+        jg.writeStringField(key, value);
+      }
+
+      @Override
+      public int write(MBeanServer mBeanServer, ObjectName qry, String attribute,
+          boolean description)
+      throws IOException {
+        return JSONBean.write(jg, mBeanServer, qry, attribute, description);
+      }
+    };
+  }
+
+  /**
+   * @param mBeanServer
+   * @param qry
+   * @param attribute
+   * @param description
+   * @return Return non-zero if failed to find bean. 0
+   * @throws IOException
+   */
+  private static int write(final JsonGenerator jg,
+      final MBeanServer mBeanServer, ObjectName qry, String attribute,
+      final boolean description)
+  throws IOException {
+    LOG.trace("Listing beans for "+qry);
+    Set<ObjectName> names = null;
+    names = mBeanServer.queryNames(qry, null);
+    jg.writeArrayFieldStart("beans");
+    Iterator<ObjectName> it = names.iterator();
+    while (it.hasNext()) {
+      ObjectName oname = it.next();
+      MBeanInfo minfo;
+      String code = "";
+      String descriptionStr = null;
+      Object attributeinfo = null;
+      try {
+        minfo = mBeanServer.getMBeanInfo(oname);
+        code = minfo.getClassName();
+        if (description) descriptionStr = minfo.getDescription();
+        String prs = "";
+        try {
+          if ("org.apache.commons.modeler.BaseModelMBean".equals(code)) {
+            prs = "modelerType";
+            code = (String) mBeanServer.getAttribute(oname, prs);
+          }
+          if (attribute != null) {
+            prs = attribute;
+            attributeinfo = mBeanServer.getAttribute(oname, prs);
+          }
+        } catch (RuntimeMBeanException e) {
+         // UnsupportedOperationExceptions happen in the normal course of business,
+         // so no need to log them as errors all the time.
+         if (e.getCause() instanceof UnsupportedOperationException) {
+           if (LOG.isTraceEnabled()) {
+             LOG.trace("Getting attribute " + prs + " of " + oname + " threw " + e);
+           }
+         } else {
+           LOG.error("Getting attribute " + prs + " of " + oname + " threw an exception", e);
+         }
+         return 0;
+        } catch (AttributeNotFoundException e) {
+          // If the modelerType attribute was not found, the class name is used
+          // instead.
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        } catch (MBeanException e) {
+          // The code inside the attribute getter threw an exception so log it,
+          // and fall back on the class name
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        } catch (RuntimeException e) {
+          // For some reason even with an MBeanException available to them
+          // Runtime exceptionscan still find their way through, so treat them
+          // the same as MBeanException
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        } catch (ReflectionException e) {
+          // This happens when the code inside the JMX bean (setter?? from the
+          // java docs) threw an exception, so log it and fall back on the
+          // class name
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        }
+      } catch (InstanceNotFoundException e) {
+        //Ignored for some reason the bean was not found so don't output it
+        continue;
+      } catch (IntrospectionException e) {
+        // This is an internal error, something odd happened with reflection so
+        // log it and don't output the bean.
+        LOG.error("Problem while trying to process JMX query: " + qry
+            + " with MBean " + oname, e);
+        continue;
+      } catch (ReflectionException e) {
+        // This happens when the code inside the JMX bean threw an exception, so
+        // log it and don't output the bean.
+        LOG.error("Problem while trying to process JMX query: " + qry
+            + " with MBean " + oname, e);
+        continue;
+      }
+
+      jg.writeStartObject();
+      jg.writeStringField("name", oname.toString());
+      if (description && descriptionStr != null && descriptionStr.length() > 0) {
+        jg.writeStringField("description", descriptionStr);
+      }
+      jg.writeStringField("modelerType", code);
+      if (attribute != null && attributeinfo == null) {
+        jg.writeStringField("result", "ERROR");
+        jg.writeStringField("message", "No attribute with name " + attribute + " was found.");
+        jg.writeEndObject();
+        jg.writeEndArray();
+        jg.close();
+        return -1;
+      }
+
+      if (attribute != null) {
+        writeAttribute(jg, attribute, descriptionStr, attributeinfo);
+      } else {
+        MBeanAttributeInfo[] attrs = minfo.getAttributes();
+        for (int i = 0; i < attrs.length; i++) {
+          writeAttribute(jg, mBeanServer, oname, description, attrs[i]);
+        }
+      }
+      jg.writeEndObject();
+    }
+    jg.writeEndArray();
+    return 0;
+  }
+
+  private static void writeAttribute(final JsonGenerator jg,
+      final MBeanServer mBeanServer, ObjectName oname,
+      final boolean description, final MBeanAttributeInfo attr)
+  throws IOException {
+    if (!attr.isReadable()) {
+      return;
+    }
+    String attName = attr.getName();
+    if ("modelerType".equals(attName)) {
+      return;
+    }
+    if (attName.indexOf("=") >= 0 || attName.indexOf(":") >= 0 || attName.indexOf(" ") >= 0) {
+      return;
+    }
+    String descriptionStr = description? attr.getDescription(): null;
+    Object value = null;
+    try {
+      value = mBeanServer.getAttribute(oname, attName);
+    } catch (RuntimeMBeanException e) {
+      // UnsupportedOperationExceptions happen in the normal course of business,
+      // so no need to log them as errors all the time.
+      if (e.getCause() instanceof UnsupportedOperationException) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Getting attribute " + attName + " of " + oname + " threw " + e);
+        }
+      } else {
+        LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      }
+      return;
+    } catch (RuntimeErrorException e) {
+      // RuntimeErrorException happens when an unexpected failure occurs in getAttribute
+      // for example https://issues.apache.org/jira/browse/DAEMON-120
+      LOG.debug("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (AttributeNotFoundException e) {
+      //Ignored the attribute was not found, which should never happen because the bean
+      //just told us that it has this attribute, but if this happens just don't output
+      //the attribute.
+      return;
+    } catch (MBeanException e) {
+      //The code inside the attribute getter threw an exception so log it, and
+      // skip outputting the attribute
+      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (RuntimeException e) {
+      //For some reason even with an MBeanException available to them Runtime exceptions
+      //can still find their way through, so treat them the same as MBeanException
+      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (ReflectionException e) {
+      //This happens when the code inside the JMX bean (setter?? from the java docs)
+      //threw an exception, so log it and skip outputting the attribute
+      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (InstanceNotFoundException e) {
+      //Ignored the mbean itself was not found, which should never happen because we
+      //just accessed it (perhaps something unregistered in-between) but if this
+      //happens just don't output the attribute.
+      return;
+    }
+
+    writeAttribute(jg, attName, descriptionStr, value);
+  }
+
+  private static void writeAttribute(JsonGenerator jg, String attName, final String descriptionStr,
+      Object value)
+  throws IOException {
+    boolean description = false;
+    if (descriptionStr != null && descriptionStr.length() > 0 && !attName.equals(descriptionStr)) {
+      description = true;
+      jg.writeFieldName(attName);
+      jg.writeStartObject();
+      jg.writeFieldName("description");
+      jg.writeString(descriptionStr);
+      jg.writeFieldName("value");
+      writeObject(jg, description, value);
+      jg.writeEndObject();
+    } else {
+      jg.writeFieldName(attName);
+      writeObject(jg, description, value);
+    }
+  }
+
+  private static void writeObject(final JsonGenerator jg, final boolean description, Object value)
+  throws IOException {
+    if(value == null) {
+      jg.writeNull();
+    } else {
+      Class<?> c = value.getClass();
+      if (c.isArray()) {
+        jg.writeStartArray();
+        int len = Array.getLength(value);
+        for (int j = 0; j < len; j++) {
+          Object item = Array.get(value, j);
+          writeObject(jg, description, item);
+        }
+        jg.writeEndArray();
+      } else if(value instanceof Number) {
+        Number n = (Number)value;
+        jg.writeNumber(n.toString());
+      } else if(value instanceof Boolean) {
+        Boolean b = (Boolean)value;
+        jg.writeBoolean(b);
+      } else if(value instanceof CompositeData) {
+        CompositeData cds = (CompositeData)value;
+        CompositeType comp = cds.getCompositeType();
+        Set<String> keys = comp.keySet();
+        jg.writeStartObject();
+        for (String key: keys) {
+          writeAttribute(jg, key, null, cds.get(key));
+        }
+        jg.writeEndObject();
+      } else if(value instanceof TabularData) {
+        TabularData tds = (TabularData)value;
+        jg.writeStartArray();
+        for(Object entry : tds.values()) {
+          writeObject(jg, description, entry);
+        }
+        jg.writeEndArray();
+      } else {
+        jg.writeString(value.toString());
+      }
+    }
+  }
+
+  /**
+   * Dump out a subset of regionserver mbeans only, not all of them, as json on System.out.
+   * @throws MalformedObjectNameException
+   * @throws IOException
+   */
+  public static String dumpRegionServerMetrics() throws MalformedObjectNameException, IOException {
+    StringWriter sw = new StringWriter(1024 * 100); // Guess this size
+    try (PrintWriter writer = new PrintWriter(sw)) {
+      JSONBean dumper = new JSONBean();
+      try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) {
+        MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
+        jsonBeanWriter.write(mbeanServer,
+          new ObjectName("java.lang:type=Memory"), null, false);
+        jsonBeanWriter.write(mbeanServer,
+          new ObjectName("Hadoop:service=HBase,name=RegionServer,sub=IPC"), null, false);
+        jsonBeanWriter.write(mbeanServer,
+          new ObjectName("Hadoop:service=HBase,name=RegionServer,sub=Replication"), null, false);
+        jsonBeanWriter.write(mbeanServer,
+          new ObjectName("Hadoop:service=HBase,name=RegionServer,sub=Server"), null, false);
+      }
+    }
+    sw.close();
+    return sw.toString();
+  }
+
+  /**
+   * Dump out all registered mbeans as json on System.out.
+   * @throws IOException
+   * @throws MalformedObjectNameException
+   */
+  public static void dumpAllBeans() throws IOException, MalformedObjectNameException {
+    try (PrintWriter writer = new PrintWriter(System.out)) {
+      JSONBean dumper = new JSONBean();
+      try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) {
+        MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
+        jsonBeanWriter.write(mbeanServer, new ObjectName("*:*"), null, false);
+      }
+    }
+  }
+
+  public static void main(String[] args) throws IOException, MalformedObjectNameException {
+    String str = dumpRegionServerMetrics();
+    System.out.println(str);
+  }
+}