You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/02/22 09:03:54 UTC

[hbase] branch master updated: HBASE-20587 Replace Jackson with shaded thirdparty gson

This is an automated email from the ASF dual-hosted git repository.

zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new f0032c9  HBASE-20587 Replace Jackson with shaded thirdparty gson
f0032c9 is described below

commit f0032c925510877396b1b0979abcc2ce83e67529
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Feb 22 16:24:22 2019 +0800

    HBASE-20587 Replace Jackson with shaded thirdparty gson
    
    Signed-off-by: Michael Stack <st...@apache.org>
---
 hbase-client/pom.xml                               |   5 -
 .../org/apache/hadoop/hbase/util/JsonMapper.java   |  16 +-
 .../apache/hadoop/hbase/client/TestOperation.java  |  23 +-
 hbase-common/pom.xml                               |   4 -
 .../org/apache/hadoop/hbase/util/GsonUtil.java     |  61 +++++
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java      |   8 +-
 .../org/apache/hadoop/hbase/util/JSONBean.java     | 271 +++++++++++----------
 .../apache/hadoop/hbase/util/JSONMetricUtil.java   |  79 ++----
 hbase-it/pom.xml                                   |   5 -
 .../apache/hadoop/hbase/RESTApiClusterManager.java | 100 ++++----
 hbase-mapreduce/pom.xml                            |  15 --
 .../hadoop/hbase/mapreduce/TableMapReduceUtil.java |   4 +-
 .../apache/hadoop/hbase/PerformanceEvaluation.java |  21 +-
 .../hadoop/hbase/TestPerformanceEvaluation.java    |  33 ++-
 .../hbase/metrics/impl/FastLongHistogram.java      |   2 +-
 hbase-rest/pom.xml                                 |  12 +
 hbase-server/pom.xml                               |  18 --
 .../apache/hadoop/hbase/io/hfile/AgeSnapshot.java  |   6 +-
 .../hadoop/hbase/io/hfile/BlockCacheUtil.java      |  78 +++---
 .../apache/hadoop/hbase/io/hfile/CacheStats.java   |  11 +-
 .../hadoop/hbase/io/hfile/LruBlockCache.java       |  34 ++-
 .../hbase/io/hfile/bucket/BucketAllocator.java     |  21 +-
 .../hadoop/hbase/io/hfile/bucket/BucketCache.java  |  21 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java     |  10 +-
 .../hadoop/hbase/monitoring/MonitoredTaskImpl.java |  14 +-
 .../apache/hadoop/hbase/wal/WALPrettyPrinter.java  |  13 +-
 .../hadoop/hbase/util/TestJSONMetricUtil.java      |  42 +---
 27 files changed, 439 insertions(+), 488 deletions(-)

diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index bb99eec..96367df 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -182,7 +182,6 @@
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-crypto</artifactId>
-      <version>${commons-crypto.version}</version>
       <exclusions>
         <exclusion>
           <groupId>net.java.dev.jna</groupId>
@@ -190,10 +189,6 @@
         </exclusion>
       </exclusions>
     </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
   </dependencies>
 
   <profiles>
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
index b5d31ff..77cbf38 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -20,10 +19,10 @@ package org.apache.hadoop.hbase.util;
 
 import java.io.IOException;
 import java.util.Map;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+
 /**
  * Utility class for converting objects to JSON
  */
@@ -32,12 +31,13 @@ public final class JsonMapper {
   private JsonMapper() {
   }
 
-  private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final Gson GSON = GsonUtil.createGson().create();
 
-  public static String writeMapAsString(Map<String, Object> map) throws IOException { 
+  public static String writeMapAsString(Map<String, Object> map) throws IOException {
     return writeObjectAsString(map);
   }
-  public static String writeObjectAsString(Object object) throws IOException { 
-    return MAPPER.writeValueAsString(object);
+
+  public static String writeObjectAsString(Object object) throws IOException {
+    return GSON.toJson(object);
   }
 }
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java
index 1875057..05596f4 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java
@@ -20,11 +20,10 @@ package org.apache.hadoop.hbase.client;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.IOException;
+import java.lang.reflect.Type;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.hadoop.hbase.Cell;
@@ -62,11 +61,15 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.BuilderStyleTest;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.GsonUtil;
 import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hbase.thirdparty.com.google.common.reflect.TypeToken;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+
 /**
  * Run tests that use the functionality of the Operation superclass for
  * Puts, Gets, Deletes, Scans, and MultiPuts.
@@ -83,7 +86,7 @@ public class TestOperation {
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static byte [] VALUE = Bytes.toBytes("testValue");
 
-  private static ObjectMapper mapper = new ObjectMapper();
+  private static Gson GSON = GsonUtil.createGson().create();
 
   private static List<Long> TS_LIST = Arrays.asList(2L, 3L, 5L);
   private static TimestampsFilter TS_FILTER = new TimestampsFilter(TS_LIST);
@@ -291,7 +294,9 @@ public class TestOperation {
     scan.addColumn(FAMILY, QUALIFIER);
     // get its JSON representation, and parse it
     String json = scan.toJSON();
-    Map<String, Object> parsedJSON = mapper.readValue(json, HashMap.class);
+    Type typeOfHashMap = new TypeToken<Map<String, Object>>() {
+    }.getType();
+    Map<String, Object> parsedJSON = GSON.fromJson(json, typeOfHashMap);
     // check for the row
     assertEquals("startRow incorrect in Scan.toJSON()",
         Bytes.toStringBinary(ROW), parsedJSON.get("startRow"));
@@ -309,7 +314,7 @@ public class TestOperation {
     get.addColumn(FAMILY, QUALIFIER);
     // get its JSON representation, and parse it
     json = get.toJSON();
-    parsedJSON = mapper.readValue(json, HashMap.class);
+    parsedJSON = GSON.fromJson(json, typeOfHashMap);
     // check for the row
     assertEquals("row incorrect in Get.toJSON()",
         Bytes.toStringBinary(ROW), parsedJSON.get("row"));
@@ -327,7 +332,7 @@ public class TestOperation {
     put.addColumn(FAMILY, QUALIFIER, VALUE);
     // get its JSON representation, and parse it
     json = put.toJSON();
-    parsedJSON = mapper.readValue(json, HashMap.class);
+    parsedJSON = GSON.fromJson(json, typeOfHashMap);
     // check for the row
     assertEquals("row absent in Put.toJSON()",
         Bytes.toStringBinary(ROW), parsedJSON.get("row"));
@@ -340,15 +345,15 @@ public class TestOperation {
     assertEquals("Qualifier incorrect in Put.toJSON()",
         Bytes.toStringBinary(QUALIFIER),
         kvMap.get("qualifier"));
-    assertEquals("Value length incorrect in Put.toJSON()",
-        VALUE.length, kvMap.get("vlen"));
+    assertEquals("Value length incorrect in Put.toJSON()", VALUE.length,
+      ((Number) kvMap.get("vlen")).intValue());
 
     // produce a Delete operation
     Delete delete = new Delete(ROW);
     delete.addColumn(FAMILY, QUALIFIER);
     // get its JSON representation, and parse it
     json = delete.toJSON();
-    parsedJSON = mapper.readValue(json, HashMap.class);
+    parsedJSON = GSON.fromJson(json, typeOfHashMap);
     // check for the row
     assertEquals("row absent in Delete.toJSON()",
         Bytes.toStringBinary(ROW), parsedJSON.get("row"));
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index 7d7deef..cdce4a9 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -239,10 +239,6 @@
       <artifactId>findbugs-annotations</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-core</artifactId>
       <scope>test</scope>
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/GsonUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/GsonUtil.java
new file mode 100644
index 0000000..80be4af
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/GsonUtil.java
@@ -0,0 +1,61 @@
+/**
+ * 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.IOException;
+import java.util.concurrent.atomic.LongAdder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.gson.GsonBuilder;
+import org.apache.hbase.thirdparty.com.google.gson.LongSerializationPolicy;
+import org.apache.hbase.thirdparty.com.google.gson.TypeAdapter;
+import org.apache.hbase.thirdparty.com.google.gson.stream.JsonReader;
+import org.apache.hbase.thirdparty.com.google.gson.stream.JsonWriter;
+
+/**
+ * Helper class for gson.
+ */
+@InterfaceAudience.Private
+public final class GsonUtil {
+
+  private GsonUtil() {
+  }
+
+  /**
+   * Create a builder which is used to create a Gson instance.
+   * <p/>
+   * Will set some common configs for the builder.
+   */
+  public static GsonBuilder createGson() {
+    return new GsonBuilder().setLongSerializationPolicy(LongSerializationPolicy.STRING)
+      .registerTypeAdapter(LongAdder.class, new TypeAdapter<LongAdder>() {
+
+        @Override
+        public void write(JsonWriter out, LongAdder value) throws IOException {
+          out.value(value.longValue());
+        }
+
+        @Override
+        public LongAdder read(JsonReader in) throws IOException {
+          LongAdder value = new LongAdder();
+          value.add(in.nextLong());
+          return value;
+        }
+      });
+  }
+}
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
index b42c270..00ea56d 100644
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
@@ -160,7 +160,6 @@ public class JMXJsonServlet extends HttpServlet {
       try {
         jsonpcb = checkCallbackName(request.getParameter(CALLBACK_PARAM));
         writer = response.getWriter();
-        beanWriter = this.jsonBeanWriter.open(writer);
 
         // "callback" parameter implies JSONP outpout
         if (jsonpcb != null) {
@@ -169,6 +168,7 @@ public class JMXJsonServlet extends HttpServlet {
         } else {
           response.setContentType("application/json; charset=utf8");
         }
+        beanWriter = this.jsonBeanWriter.open(writer);
         // Should we output description on each attribute and bean?
         String tmpStr = request.getParameter(INCLUDE_DESCRIPTION);
         boolean description = tmpStr != null && tmpStr.length() > 0;
@@ -202,9 +202,11 @@ public class JMXJsonServlet extends HttpServlet {
           response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
         }
       } finally {
-        if (beanWriter != null) beanWriter.close();
+        if (beanWriter != null) {
+          beanWriter.close();
+        }
         if (jsonpcb != null) {
-           writer.write(");");
+          writer.write(");");
         }
         if (writer != null) {
           writer.close();
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
similarity index 53%
rename from hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
rename to hbase-http/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
index da89a41..c9b18e3 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
@@ -16,9 +16,6 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonGenerationException;
-import com.fasterxml.jackson.core.JsonGenerator;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
@@ -47,54 +44,78 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.stream.JsonWriter;
+
 /**
  * Utility for doing JSON and MBeans.
  */
 @InterfaceAudience.Private
 public class JSONBean {
   private static final Logger LOG = LoggerFactory.getLogger(JSONBean.class);
-  private final JsonFactory jsonFactory;
-
-  public JSONBean() {
-    this.jsonFactory = new JsonFactory();
-  }
+  private static final Gson GSON = GsonUtil.createGson().create();
 
   /**
    * 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 write(String key, String value) throws IOException;
+
+    int write(MBeanServer mBeanServer, ObjectName qry, String attribute, boolean description)
+        throws IOException;
+
     void flush() throws IOException;
   }
 
+  /**
+   * Notice that, closing the return {@link Writer} will not close the {@code writer} passed in, you
+   * still need to close the {@code writer} by yourself.
+   * <p/>
+   * This is because that, we can only finish the json after you call {@link Writer#close()}. So if
+   * we just close the {@code writer}, you can write nothing after finished the json.
+   */
   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();
+    JsonWriter jsonWriter = GSON.newJsonWriter(new java.io.Writer() {
+
+      @Override
+      public void write(char[] cbuf, int off, int len) throws IOException {
+        writer.write(cbuf, off, len);
+      }
+
+      @Override
+      public void flush() throws IOException {
+        writer.flush();
+      }
+
+      @Override
+      public void close() throws IOException {
+        // do nothing
+      }
+    });
+    jsonWriter.setIndent("  ");
+    jsonWriter.beginObject();
     return new Writer() {
       @Override
       public void flush() throws IOException {
-        jg.flush();
+        jsonWriter.flush();
       }
 
       @Override
       public void close() throws IOException {
-        jg.close();
+        jsonWriter.endObject();
+        jsonWriter.close();
       }
 
       @Override
-      public void write(String key, String value) throws JsonGenerationException, IOException {
-        jg.writeStringField(key, value);
+      public void write(String key, String value) throws IOException {
+        jsonWriter.name(key).value(value);
       }
 
       @Override
       public int write(MBeanServer mBeanServer, ObjectName qry, String attribute,
-          boolean description)
-      throws IOException {
-        return JSONBean.write(jg, mBeanServer, qry, attribute, description);
+          boolean description) throws IOException {
+        return JSONBean.write(jsonWriter, mBeanServer, qry, attribute, description);
       }
     };
   }
@@ -102,14 +123,12 @@ public class JSONBean {
   /**
    * @return Return non-zero if failed to find bean. 0
    */
-  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);
+  private static int write(JsonWriter writer, MBeanServer mBeanServer, ObjectName qry,
+      String attribute, boolean description) throws IOException {
+    LOG.trace("Listing beans for " + qry);
     Set<ObjectName> names = null;
     names = mBeanServer.queryNames(qry, null);
-    jg.writeArrayFieldStart("beans");
+    writer.name("beans").beginArray();
     Iterator<ObjectName> it = names.iterator();
     while (it.hasNext()) {
       ObjectName oname = it.next();
@@ -120,7 +139,9 @@ public class JSONBean {
       try {
         minfo = mBeanServer.getMBeanInfo(oname);
         code = minfo.getClassName();
-        if (description) descriptionStr = minfo.getDescription();
+        if (description) {
+          descriptionStr = minfo.getDescription();
+        }
         String prs = "";
         try {
           if ("org.apache.commons.modeler.BaseModelMBean".equals(code)) {
@@ -132,89 +153,80 @@ public class JSONBean {
             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;
+          // 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);
+          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);
+          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);
+          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);
+          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
+        // 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);
+        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);
+        LOG.error("Problem while trying to process JMX query: " + qry + " with MBean " + oname, e);
         continue;
       }
-
-      jg.writeStartObject();
-      jg.writeStringField("name", oname.toString());
+      writer.beginObject();
+      writer.name("name").value(oname.toString());
       if (description && descriptionStr != null && descriptionStr.length() > 0) {
-        jg.writeStringField("description", descriptionStr);
+        writer.name("description").value(descriptionStr);
       }
-      jg.writeStringField("modelerType", code);
+      writer.name("modelerType").value(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();
+        writer.name("result").value("ERROR");
+        writer.name("message").value("No attribute with name " + attribute + " was found.");
+        writer.endObject();
+        writer.endArray();
+        writer.close();
         return -1;
       }
 
       if (attribute != null) {
-        writeAttribute(jg, attribute, descriptionStr, attributeinfo);
+        writeAttribute(writer, attribute, descriptionStr, attributeinfo);
       } else {
         MBeanAttributeInfo[] attrs = minfo.getAttributes();
         for (int i = 0; i < attrs.length; i++) {
-          writeAttribute(jg, mBeanServer, oname, description, attrs[i]);
+          writeAttribute(writer, mBeanServer, oname, description, attrs[i]);
         }
       }
-      jg.writeEndObject();
+      writer.endObject();
     }
-    jg.writeEndArray();
+    writer.endArray();
     return 0;
   }
 
-  private static void writeAttribute(final JsonGenerator jg,
-      final MBeanServer mBeanServer, ObjectName oname,
-      final boolean description, final MBeanAttributeInfo attr)
-  throws IOException {
+  private static void writeAttribute(JsonWriter writer, MBeanServer mBeanServer, ObjectName oname,
+      boolean description, MBeanAttributeInfo attr) throws IOException {
     if (!attr.isReadable()) {
       return;
     }
@@ -225,7 +237,7 @@ public class JSONBean {
     if (attName.indexOf("=") >= 0 || attName.indexOf(":") >= 0 || attName.indexOf(" ") >= 0) {
       return;
     }
-    String descriptionStr = description? attr.getDescription(): null;
+    String descriptionStr = description ? attr.getDescription() : null;
     Object value = null;
     try {
       value = mBeanServer.getAttribute(oname, attName);
@@ -237,117 +249,110 @@ public class JSONBean {
           LOG.trace("Getting attribute " + attName + " of " + oname + " threw " + e);
         }
       } else {
-        LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+        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);
+      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.
+      // 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
+      // 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);
+      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);
+      // 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);
+      // 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.
+      // 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);
+    writeAttribute(writer, attName, descriptionStr, value);
   }
 
-  private static void writeAttribute(JsonGenerator jg, String attName, final String descriptionStr,
-      Object value)
-  throws IOException {
-    boolean description = false;
+  private static void writeAttribute(JsonWriter writer, String attName, String descriptionStr,
+      Object value) throws IOException {
     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();
+      writer.name(attName);
+      writer.beginObject();
+      writer.name("description").value(descriptionStr);
+      writer.name("value");
+      writeObject(writer, value);
+      writer.endObject();
     } else {
-      jg.writeFieldName(attName);
-      writeObject(jg, description, value);
+      writer.name(attName);
+      writeObject(writer, value);
     }
   }
 
-  private static void writeObject(final JsonGenerator jg, final boolean description, Object value)
-  throws IOException {
-    if(value == null) {
-      jg.writeNull();
+  private static void writeObject(JsonWriter writer, Object value) throws IOException {
+    if (value == null) {
+      writer.nullValue();
     } else {
       Class<?> c = value.getClass();
       if (c.isArray()) {
-        jg.writeStartArray();
+        writer.beginArray();
         int len = Array.getLength(value);
         for (int j = 0; j < len; j++) {
           Object item = Array.get(value, j);
-          writeObject(jg, description, item);
+          writeObject(writer, item);
         }
-        jg.writeEndArray();
-      } else if(value instanceof Number) {
-        Number n = (Number)value;
+        writer.endArray();
+      } else if (value instanceof Number) {
+        Number n = (Number) value;
         if (Double.isFinite(n.doubleValue())) {
-          jg.writeNumber(n.toString());
+          writer.value(n);
         } else {
-          jg.writeString(n.toString());
+          writer.value(n.toString());
         }
-      } else if(value instanceof Boolean) {
-        Boolean b = (Boolean)value;
-        jg.writeBoolean(b);
-      } else if(value instanceof CompositeData) {
-        CompositeData cds = (CompositeData)value;
+      } else if (value instanceof Boolean) {
+        Boolean b = (Boolean) value;
+        writer.value(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));
+        writer.beginObject();
+        for (String key : keys) {
+          writeAttribute(writer, 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);
+        writer.endObject();
+      } else if (value instanceof TabularData) {
+        TabularData tds = (TabularData) value;
+        writer.beginArray();
+        for (Object entry : tds.values()) {
+          writeObject(writer, entry);
         }
-        jg.writeEndArray();
+        writer.endArray();
       } else {
-        jg.writeString(value.toString());
+        writer.value(value.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(
-        new OutputStreamWriter(System.out, StandardCharsets.UTF_8))) {
+    try (PrintWriter writer =
+      new PrintWriter(new OutputStreamWriter(System.out, StandardCharsets.UTF_8))) {
       JSONBean dumper = new JSONBean();
       try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) {
         MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
similarity index 72%
rename from hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
rename to hbase-http/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
index 7bc2257e..6e155ae 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
@@ -17,9 +17,6 @@
  * */
 package org.apache.hadoop.hbase.util;
 
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import java.beans.IntrospectionException;
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -49,22 +46,22 @@ public final class JSONMetricUtil {
   private static final Logger LOG = LoggerFactory.getLogger(JSONMetricUtil.class);
 
   private static MBeanServer mbServer = ManagementFactory.getPlatformMBeanServer();
-  //MBeans ObjectName domain names
+  // MBeans ObjectName domain names
   public static final String JAVA_LANG_DOMAIN = "java.lang";
   public static final String JAVA_NIO_DOMAIN = "java.nio";
   public static final String SUN_MGMT_DOMAIN = "com.sun.management";
   public static final String HADOOP_DOMAIN = "Hadoop";
 
-  //MBeans ObjectName properties key names
+  // MBeans ObjectName properties key names
   public static final String TYPE_KEY = "type";
   public static final String NAME_KEY = "name";
   public static final String SERVICE_KEY = "service";
   public static final String SUBSYSTEM_KEY = "sub";
 
-/**
- * Utility for getting metric values. Collection of static methods intended for
- * easier access to metric values.
- */
+  /**
+   * Utility for getting metric values. Collection of static methods intended for easier access to
+   * metric values.
+   */
   private JSONMetricUtil() {
     // Not to be called
   }
@@ -80,70 +77,39 @@ public final class JSONMetricUtil {
     Object value = null;
     try {
       value = mbServer.getAttribute(bean, attribute);
-    }
-    catch(Exception e) {
-      LOG.error("Unable to get value from MBean= "+ bean.toString() +
-        "for attribute=" + attribute + " " + e.getMessage());
+    } catch (Exception e) {
+      LOG.error("Unable to get value from MBean= " + bean.toString() + "for attribute=" +
+        attribute + " " + e.getMessage());
     }
     return value;
   }
 
   /**
-   * Returns a subset of mbeans defined by qry.
-   * Modeled after DumpRegionServerMetrics#dumpMetrics.
+   * Returns a subset of mbeans defined by qry. Modeled after DumpRegionServerMetrics#dumpMetrics.
    * Example: String qry= "java.lang:type=Memory"
    * @throws MalformedObjectNameException if json have bad format
    * @throws IOException /
    * @return String representation of json array.
    */
-  public static String dumpBeanToString(String qry) throws MalformedObjectNameException,
-  IOException {
+  public static String dumpBeanToString(String qry)
+      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(qry), null, false);
+        jsonBeanWriter.write(mbeanServer, new ObjectName(qry), null, false);
       }
     }
     sw.close();
     return sw.toString();
   }
 
-  public static JsonNode mappStringToJsonNode(String jsonString)
-      throws JsonProcessingException, IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    JsonNode node = mapper.readTree(jsonString);
-    return node;
-  }
-
-
-  public static JsonNode searchJson(JsonNode tree, String searchKey)
-      throws JsonProcessingException, IOException {
-    if (tree == null) {
-      return null;
-    }
-    if(tree.has(searchKey)) {
-      return tree.get(searchKey);
-    }
-    if(tree.isContainerNode()) {
-      for(JsonNode branch: tree) {
-        JsonNode branchResult = searchJson(branch, searchKey);
-        if (branchResult != null && !branchResult.isMissingNode()) {
-          return branchResult;
-        }
-      }
-    }
-    return null;
-  }
-
   /**
-   * Method for building hashtable used for constructing ObjectName.
-   * Mapping is done with arrays indices
-   * @param keys Hashtable keys
-   * @param values Hashtable values
-   * @return Hashtable or null if arrays are empty * or have different number of elements
+   * Method for building map used for constructing ObjectName. Mapping is done with arrays indices
+   * @param keys Map keys
+   * @param values Map values
+   * @return Map or null if arrays are empty * or have different number of elements
    */
   public static Hashtable<String, String> buldKeyValueTable(String[] keys, String[] values) {
     if (keys.length != values.length) {
@@ -154,8 +120,8 @@ public final class JSONMetricUtil {
       LOG.error("keys and values arrays can not be empty;");
       return null;
     }
-    Hashtable<String, String> table = new Hashtable<String, String>();
-    for(int i = 0; i < keys.length; i++) {
+    Hashtable<String, String> table = new Hashtable<>();
+    for (int i = 0; i < keys.length; i++) {
       table.put(keys[i], values[i]);
     }
     return table;
@@ -178,8 +144,7 @@ public final class JSONMetricUtil {
     return ManagementFactory.getRuntimeMXBean().getName().split("@")[0];
   }
 
-  public static String getCommmand() throws MalformedObjectNameException,
-  IOException {
+  public static String getCommmand() throws MalformedObjectNameException, IOException {
     RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
     return runtimeBean.getSystemProperties().get("sun.java.command");
   }
@@ -193,7 +158,7 @@ public final class JSONMetricUtil {
     long lastGcDuration = 0;
     Object lastGcInfo = getValueFromMBean(gcCollector, "LastGcInfo");
     if (lastGcInfo != null && lastGcInfo instanceof CompositeData) {
-      CompositeData cds = (CompositeData)lastGcInfo;
+      CompositeData cds = (CompositeData) lastGcInfo;
       lastGcDuration = (long) cds.get("duration");
     }
     return lastGcDuration;
@@ -208,6 +173,6 @@ public final class JSONMetricUtil {
     if (a == 0 || b == 0) {
       return 0;
     }
-    return ((float)a / (float)b) *100;
+    return ((float) a / (float) b) * 100;
   }
 }
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index ff65e81..956b87e 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -222,7 +222,6 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-backup</artifactId>
-      <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase.thirdparty</groupId>
@@ -279,10 +278,6 @@
       <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
   </dependencies>
 
   <profiles>
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
index 1cd8147..ae16af4 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
@@ -18,16 +18,11 @@
 
 package org.apache.hadoop.hbase;
 
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.glassfish.jersey.client.authentication.HttpAuthenticationFeature;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
 import javax.ws.rs.client.Client;
 import javax.ws.rs.client.ClientBuilder;
 import javax.ws.rs.client.Entity;
@@ -37,11 +32,17 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriBuilder;
 import javax.xml.ws.http.HTTPException;
-import java.io.IOException;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.Locale;
-import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.glassfish.jersey.client.authentication.HttpAuthenticationFeature;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
+import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
 
 /**
  * A ClusterManager implementation designed to control Cloudera Manager (http://www.cloudera.com)
@@ -76,6 +77,8 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
   private static final String REST_API_CLUSTER_MANAGER_CLUSTER_NAME =
       "hbase.it.clustermanager.restapi.clustername";
 
+  private static final Gson GSON = GsonUtil.createGson().create();
+
   // Some default values for the above properties.
   private static final String DEFAULT_SERVER_HOSTNAME = "http://localhost:7180";
   private static final String DEFAULT_SERVER_USERNAME = "admin";
@@ -215,17 +218,14 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
   private String getHostId(String hostname) throws IOException {
     String hostId = null;
 
-    URI uri = UriBuilder.fromUri(serverHostname)
-        .path("api")
-        .path(API_VERSION)
-        .path("hosts")
-        .build();
-    JsonNode hosts = getJsonNodeFromURIGet(uri);
+    URI uri =
+      UriBuilder.fromUri(serverHostname).path("api").path(API_VERSION).path("hosts").build();
+    JsonElement hosts = getJsonNodeFromURIGet(uri);
     if (hosts != null) {
       // Iterate through the list of hosts, stopping once you've reached the requested hostname.
-      for (JsonNode host : hosts) {
-        if (host.get("hostname").textValue().equals(hostname)) {
-          hostId = host.get("hostId").textValue();
+      for (JsonElement host : hosts.getAsJsonArray()) {
+        if (host.getAsJsonObject().get("hostname").getAsString().equals(hostname)) {
+          hostId = host.getAsJsonObject().get("hostId").getAsString();
           break;
         }
       }
@@ -237,18 +237,17 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
   }
 
   // Execute GET against URI, returning a JsonNode object to be traversed.
-  private JsonNode getJsonNodeFromURIGet(URI uri) throws IOException {
+  private JsonElement getJsonNodeFromURIGet(URI uri) throws IOException {
     LOG.info("Executing GET against " + uri + "...");
     WebTarget webTarget = client.target(uri);
-    Invocation.Builder invocationBuilder =  webTarget.request(MediaType.APPLICATION_JSON);
+    Invocation.Builder invocationBuilder = webTarget.request(MediaType.APPLICATION_JSON);
     Response response = invocationBuilder.get();
     int statusCode = response.getStatus();
     if (statusCode != Response.Status.OK.getStatusCode()) {
       throw new HTTPException(statusCode);
     }
     // This API folds information as the value to an "items" attribute.
-    return new ObjectMapper().readTree(response.readEntity(String.class)).get("items");
-
+    return GSON.toJsonTree(response.readEntity(String.class)).getAsJsonObject().get("items");
   }
 
   // This API assigns a unique role name to each host's instance of a role.
@@ -257,29 +256,21 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
     return getRolePropertyValue(serviceName, roleType, hostId, "name");
   }
 
-  // Get the value of a  property from a role on a particular host.
+  // Get the value of a property from a role on a particular host.
   private String getRolePropertyValue(String serviceName, String roleType, String hostId,
       String property) throws IOException {
     String roleValue = null;
-    URI uri = UriBuilder.fromUri(serverHostname)
-        .path("api")
-        .path(API_VERSION)
-        .path("clusters")
-        .path(clusterName)
-        .path("services")
-        .path(serviceName)
-        .path("roles")
-        .build();
-    JsonNode roles = getJsonNodeFromURIGet(uri);
+    URI uri = UriBuilder.fromUri(serverHostname).path("api").path(API_VERSION).path("clusters")
+      .path(clusterName).path("services").path(serviceName).path("roles").build();
+    JsonElement roles = getJsonNodeFromURIGet(uri);
     if (roles != null) {
       // Iterate through the list of roles, stopping once the requested one is found.
-      for (JsonNode role : roles) {
-        if (role.get("hostRef").get("hostId").textValue().equals(hostId) &&
-            role.get("type")
-                .textValue()
-                .toLowerCase(Locale.ROOT)
-                .equals(roleType.toLowerCase(Locale.ROOT))) {
-          roleValue = role.get(property).textValue();
+      for (JsonElement role : roles.getAsJsonArray()) {
+        JsonObject roleObj = role.getAsJsonObject();
+        if (roleObj.get("hostRef").getAsJsonObject().get("hostId").getAsString().equals(hostId) &&
+          roleObj.get("type").getAsString().toLowerCase(Locale.ROOT)
+            .equals(roleType.toLowerCase(Locale.ROOT))) {
+          roleValue = roleObj.get(property).getAsString();
           break;
         }
       }
@@ -297,19 +288,14 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
   // Convert a service (e.g. "HBASE," "HDFS") into a service name (e.g. "HBASE-1," "HDFS-1").
   private String getServiceName(Service service) throws IOException {
     String serviceName = null;
-    URI uri = UriBuilder.fromUri(serverHostname)
-        .path("api")
-        .path(API_VERSION)
-        .path("clusters")
-        .path(clusterName)
-        .path("services")
-        .build();
-    JsonNode services = getJsonNodeFromURIGet(uri);
+    URI uri = UriBuilder.fromUri(serverHostname).path("api").path(API_VERSION).path("clusters")
+      .path(clusterName).path("services").build();
+    JsonElement services = getJsonNodeFromURIGet(uri);
     if (services != null) {
       // Iterate through the list of services, stopping once the requested one is found.
-      for (JsonNode serviceEntry : services) {
-        if (serviceEntry.get("type").textValue().equals(service.toString())) {
-          serviceName = serviceEntry.get("name").textValue();
+      for (JsonElement serviceEntry : services.getAsJsonArray()) {
+        if (serviceEntry.getAsJsonObject().get("type").getAsString().equals(service.toString())) {
+          serviceName = serviceEntry.getAsJsonObject().get("name").getAsString();
           break;
         }
       }
diff --git a/hbase-mapreduce/pom.xml b/hbase-mapreduce/pom.xml
index 2bf693b..19c2e4f 100644
--- a/hbase-mapreduce/pom.xml
+++ b/hbase-mapreduce/pom.xml
@@ -133,7 +133,6 @@
            Snapshot protos. TODO: Move to internal types.-->
       <groupId>com.google.protobuf</groupId>
       <artifactId>protobuf-java</artifactId>
-      <version>${external.protobuf.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
@@ -255,19 +254,6 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
-    <!-- jackson(s) used by PerformanceEvaluation and it looks like TableMapReduceUtil -->
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
-    </dependency>
   </dependencies>
   <profiles>
     <!-- Skip the tests in this module -->
@@ -369,7 +355,6 @@
               <artifactId>xercesImpl</artifactId>
             </exclusion>
           </exclusions>
-          <version>${hadoop-two.version}</version>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index ce9f93a..ad189c6 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -819,9 +819,7 @@ public class TableMapReduceUtil {
       org.apache.htrace.core.Tracer.class,
       com.codahale.metrics.MetricRegistry.class,
       org.apache.commons.lang3.ArrayUtils.class,
-      com.fasterxml.jackson.databind.ObjectMapper.class,
-      com.fasterxml.jackson.core.Versioned.class,
-      com.fasterxml.jackson.annotation.JsonView.class,
+      org.apache.hbase.thirdparty.com.google.gson.Gson.class,
       org.apache.hadoop.hbase.zookeeper.ZKWatcher.class);
   }
 
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 04f5aae..85897e9 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -20,9 +20,6 @@ package org.apache.hadoop.hbase;
 
 import com.codahale.metrics.Histogram;
 import com.codahale.metrics.UniformReservoir;
-import com.fasterxml.jackson.databind.MapperFeature;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
 import java.io.IOException;
 import java.io.PrintStream;
 import java.lang.reflect.Constructor;
@@ -45,7 +42,6 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -89,6 +85,7 @@ import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.ByteArrayHashKey;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.GsonUtil;
 import org.apache.hadoop.hbase.util.Hash;
 import org.apache.hadoop.hbase.util.MurmurHash;
 import org.apache.hadoop.hbase.util.Pair;
@@ -108,8 +105,10 @@ import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
 
 /**
  * Script used evaluating HBase performance and scalability.  Runs a HBase
@@ -134,10 +133,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
   static final String RANDOM_READ = "randomRead";
   static final String PE_COMMAND_SHORTNAME = "pe";
   private static final Logger LOG = LoggerFactory.getLogger(PerformanceEvaluation.class.getName());
-  private static final ObjectMapper MAPPER = new ObjectMapper();
-  static {
-    MAPPER.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true);
-  }
+  private static final Gson GSON = GsonUtil.createGson().create();
 
   public static final String TABLE_NAME = "TestTable";
   public static final String FAMILY_NAME_BASE = "info";
@@ -308,8 +304,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         }
       };
 
-      ObjectMapper mapper = new ObjectMapper();
-      TestOptions opts = mapper.readValue(value.toString(), TestOptions.class);
+      TestOptions opts = GSON.fromJson(value.toString(), TestOptions.class);
       Configuration conf = HBaseConfiguration.create(context.getConfiguration());
       final Connection con = ConnectionFactory.createConnection(conf);
       AsyncConnection asyncCon = null;
@@ -566,7 +561,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     TableMapReduceUtil.addDependencyJars(job);
     TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
       Histogram.class,     // yammer metrics
-      ObjectMapper.class,  // jackson-mapper-asl
+      Gson.class,  // gson
       FilterAllFilter.class // hbase-server tests jar
       );
 
@@ -612,7 +607,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         TestOptions next = new TestOptions(opts);
         next.startRow = j * perClientRows;
         next.perClientRunRows = perClientRows;
-        String s = MAPPER.writeValueAsString(next);
+        String s = GSON.toJson(next);
         LOG.info("Client=" + j + ", input=" + s);
         byte[] b = Bytes.toBytes(s);
         int hash = h.hash(new ByteArrayHashKey(b, 0, b.length), -1);
@@ -2374,7 +2369,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
       InterruptedException, ClassNotFoundException, ExecutionException {
     // Log the configuration we're going to run with. Uses JSON mapper because lazy. It'll do
     // the TestOptions introspection for us and dump the output in a readable format.
-    LOG.info(cmd.getSimpleName() + " test run options=" + MAPPER.writeValueAsString(opts));
+    LOG.info(cmd.getSimpleName() + " test run options=" + GSON.toJson(opts));
     Admin admin = null;
     Connection connection = null;
     try {
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
index dfcf2d6..bcaa975 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
@@ -17,14 +17,14 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import com.codahale.metrics.Histogram;
 import com.codahale.metrics.Snapshot;
 import com.codahale.metrics.UniformReservoir;
-import com.fasterxml.jackson.core.JsonGenerationException;
-import com.fasterxml.jackson.databind.JsonMappingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
@@ -44,11 +44,13 @@ import org.apache.hadoop.hbase.PerformanceEvaluation.TestOptions;
 import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.GsonUtil;
 import org.junit.ClassRule;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+
 @Category({MiscTests.class, SmallTests.class})
 public class TestPerformanceEvaluation {
 
@@ -72,23 +74,21 @@ public class TestPerformanceEvaluation {
   }
 
   @Test
-  public void testSerialization()
-  throws JsonGenerationException, JsonMappingException, IOException {
+  public void testSerialization() throws IOException {
     PerformanceEvaluation.TestOptions options = new PerformanceEvaluation.TestOptions();
     assertTrue(!options.isAutoFlush());
     options.setAutoFlush(true);
-    ObjectMapper mapper = new ObjectMapper();
-    String optionsString = mapper.writeValueAsString(options);
+    Gson gson = GsonUtil.createGson().create();
+    String optionsString = gson.toJson(options);
     PerformanceEvaluation.TestOptions optionsDeserialized =
-        mapper.readValue(optionsString, PerformanceEvaluation.TestOptions.class);
+      gson.fromJson(optionsString, PerformanceEvaluation.TestOptions.class);
     assertTrue(optionsDeserialized.isAutoFlush());
   }
 
   /**
-   * Exercise the mr spec writing.  Simple assertions to make sure it is basically working.
-   * @throws IOException
+   * Exercise the mr spec writing. Simple assertions to make sure it is basically working.
    */
-  @Ignore @Test
+  @Test
   public void testWriteInputFile() throws IOException {
     TestOptions opts = new PerformanceEvaluation.TestOptions();
     final int clients = 10;
@@ -100,13 +100,12 @@ public class TestPerformanceEvaluation {
     Path p = new Path(dir, PerformanceEvaluation.JOB_INPUT_FILENAME);
     long len = fs.getFileStatus(p).getLen();
     assertTrue(len > 0);
-    byte [] content = new byte[(int)len];
+    byte[] content = new byte[(int) len];
     FSDataInputStream dis = fs.open(p);
     try {
       dis.readFully(content);
-      BufferedReader br =
-        new BufferedReader(new InputStreamReader(
-              new ByteArrayInputStream(content), StandardCharsets.UTF_8));
+      BufferedReader br = new BufferedReader(
+        new InputStreamReader(new ByteArrayInputStream(content), StandardCharsets.UTF_8));
       int count = 0;
       while (br.readLine() != null) {
         count++;
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
index 69f4ae5..8154460 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
@@ -21,7 +21,6 @@ import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.stream.Stream;
-
 import org.apache.hadoop.hbase.metrics.Snapshot;
 import org.apache.hadoop.hbase.util.AtomicUtils;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -47,6 +46,7 @@ public class FastLongHistogram {
    * Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
    */
   private static class Bins {
+
     private final LongAdder[] counts;
     // inclusive
     private final long binsMin;
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index d06feec..3f27038 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -300,6 +300,18 @@
       <artifactId>jackson-jaxrs-json-provider</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
       <!-- We *might* need this for XMLStreamReader use in RemoteAdmin
            TODO figure out if we can remove it.
         -->
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index fff0a3a..d101e89 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -70,7 +70,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-remote-resources-plugin</artifactId>
-        <version>1.5</version>
         <dependencies>
           <!-- resource bundle only needed at build time -->
           <dependency>
@@ -500,22 +499,6 @@
       <groupId>javax.servlet</groupId>
       <artifactId>javax.servlet-api</artifactId>
     </dependency>
-    <!-- Jackson only used in compile/runtime scope by BlockCacheUtil class
-         also used by some tests
-      -->
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
-    </dependency>
-
     <!-- tracing Dependencies -->
     <dependency>
       <groupId>org.apache.htrace</groupId>
@@ -579,7 +562,6 @@
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-crypto</artifactId>
-      <version>${commons-crypto.version}</version>
       <exclusions>
         <exclusion>
           <groupId>net.java.dev.jna</groupId>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
index 054d54b..cd89322 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import org.apache.hadoop.hbase.metrics.impl.FastLongHistogram;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -26,11 +25,10 @@ import org.apache.yetus.audience.InterfaceAudience;
  * This object is preferred because we can control how it is serialized out when JSON'ing.
  */
 @InterfaceAudience.Private
-@JsonIgnoreProperties({"ageHistogram", "snapshot"})
 public class AgeSnapshot {
 
-  private final FastLongHistogram ageHistogram;
-  private final long[] quantiles;
+  private transient final FastLongHistogram ageHistogram;
+  private transient final long[] quantiles;
 
   AgeSnapshot(final FastLongHistogram ageHistogram) {
     this.ageHistogram = ageHistogram;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
index 36f9e61..cce7972 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
@@ -23,19 +23,19 @@ import java.util.NavigableMap;
 import java.util.NavigableSet;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.ConcurrentSkipListSet;
-
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-import com.fasterxml.jackson.core.JsonGenerationException;
-import com.fasterxml.jackson.databind.JsonMappingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializationFeature;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.metrics.impl.FastLongHistogram;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.TypeAdapter;
+import org.apache.hbase.thirdparty.com.google.gson.stream.JsonReader;
+import org.apache.hbase.thirdparty.com.google.gson.stream.JsonWriter;
+
 /**
  * Utilty for aggregating counts in CachedBlocks and toString/toJSON CachedBlocks and BlockCaches.
  * No attempt has been made at making this thread safe.
@@ -50,12 +50,29 @@ public class BlockCacheUtil {
   /**
    * Needed generating JSON.
    */
-  private static final ObjectMapper MAPPER = new ObjectMapper();
-  static {
-    MAPPER.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
-    MAPPER.configure(SerializationFeature.FLUSH_AFTER_WRITE_VALUE, true);
-    MAPPER.configure(SerializationFeature.INDENT_OUTPUT, true);
-  }
+  private static final Gson GSON = GsonUtil.createGson()
+    .registerTypeAdapter(FastLongHistogram.class, new TypeAdapter<FastLongHistogram>() {
+
+      @Override
+      public void write(JsonWriter out, FastLongHistogram value) throws IOException {
+        AgeSnapshot snapshot = new AgeSnapshot(value);
+        out.beginObject();
+        out.name("mean").value(snapshot.getMean());
+        out.name("min").value(snapshot.getMin());
+        out.name("max").value(snapshot.getMax());
+        out.name("75thPercentile").value(snapshot.get75thPercentile());
+        out.name("95thPercentile").value(snapshot.get95thPercentile());
+        out.name("98thPercentile").value(snapshot.get98thPercentile());
+        out.name("99thPercentile").value(snapshot.get99thPercentile());
+        out.name("999thPercentile").value(snapshot.get999thPercentile());
+        out.endObject();
+      }
+
+      @Override
+      public FastLongHistogram read(JsonReader in) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+    }).setPrettyPrinting().create();
 
   /**
    * @param cb
@@ -102,17 +119,12 @@ public class BlockCacheUtil {
   }
 
   /**
-   * @param filename
-   * @param blocks
    * @return A JSON String of <code>filename</code> and counts of <code>blocks</code>
-   * @throws JsonGenerationException
-   * @throws JsonMappingException
-   * @throws IOException
    */
-  public static String toJSON(final String filename, final NavigableSet<CachedBlock> blocks)
-  throws JsonGenerationException, JsonMappingException, IOException {
+  public static String toJSON(String filename, NavigableSet<CachedBlock> blocks)
+      throws IOException {
     CachedBlockCountsPerFile counts = new CachedBlockCountsPerFile(filename);
-    for (CachedBlock cb: blocks) {
+    for (CachedBlock cb : blocks) {
       counts.count++;
       counts.size += cb.getSize();
       BlockType bt = cb.getBlockType();
@@ -121,31 +133,21 @@ public class BlockCacheUtil {
         counts.sizeData += cb.getSize();
       }
     }
-    return MAPPER.writeValueAsString(counts);
+    return GSON.toJson(counts);
   }
 
   /**
-   * @param cbsbf
    * @return JSON string of <code>cbsf</code> aggregated
-   * @throws JsonGenerationException
-   * @throws JsonMappingException
-   * @throws IOException
    */
-  public static String toJSON(final CachedBlocksByFile cbsbf)
-  throws JsonGenerationException, JsonMappingException, IOException {
-    return MAPPER.writeValueAsString(cbsbf);
+  public static String toJSON(CachedBlocksByFile cbsbf) throws IOException {
+    return GSON.toJson(cbsbf);
   }
 
   /**
-   * @param bc
    * @return JSON string of <code>bc</code> content.
-   * @throws JsonGenerationException
-   * @throws JsonMappingException
-   * @throws IOException
    */
-  public static String toJSON(final BlockCache bc)
-  throws JsonGenerationException, JsonMappingException, IOException {
-    return MAPPER.writeValueAsString(bc);
+  public static String toJSON(BlockCache bc) throws IOException {
+    return GSON.toJson(bc);
   }
 
   /**
@@ -254,7 +256,6 @@ public class BlockCacheUtil {
    * This is different than metrics in that it is stats on current state of a cache.
    * See getLoadedCachedBlocksByFile
    */
-  @JsonIgnoreProperties({"cachedBlockStatsByFile"})
   public static class CachedBlocksByFile {
     private int count;
     private int dataBlockCount;
@@ -282,7 +283,8 @@ public class BlockCacheUtil {
     /**
      * Map by filename. use concurent utils because we want our Map and contained blocks sorted.
      */
-    private NavigableMap<String, NavigableSet<CachedBlock>> cachedBlockByFile = new ConcurrentSkipListMap<>();
+    private transient NavigableMap<String, NavigableSet<CachedBlock>> cachedBlockByFile =
+      new ConcurrentSkipListMap<>();
     FastLongHistogram hist = new FastLongHistogram();
 
     /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
index c1c92e1..7c5b563 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
@@ -21,9 +21,8 @@ package org.apache.hadoop.hbase.io.hfile;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
-
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.impl.FastLongHistogram;
+import org.apache.yetus.audience.InterfaceAudience;
 
 
 /**
@@ -100,13 +99,13 @@ public class CacheStats {
   /** The number of metrics periods to include in window */
   private final int numPeriodsInWindow;
   /** Hit counts for each period in window */
-  private final long [] hitCounts;
+  private final long[] hitCounts;
   /** Caching hit counts for each period in window */
-  private final long [] hitCachingCounts;
+  private final long[] hitCachingCounts;
   /** Access counts for each period in window */
-  private final long [] requestCounts;
+  private final long[] requestCounts;
   /** Caching access counts for each period in window */
-  private final long [] requestCachingCounts;
+  private final long[] requestCachingCounts;
   /** Last hit count read */
   private long lastHitCount = 0;
   /** Last hit caching count read */
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
index 1dab053..c21935f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -33,26 +32,22 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.concurrent.locks.ReentrantLock;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
 import org.apache.hbase.thirdparty.com.google.common.base.Objects;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-
 /**
  * A block cache implementation that is memory-aware using {@link HeapSize},
  * memory-bound using an LRU eviction algorithm, and concurrent: backed by a
@@ -96,7 +91,6 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
  * to the relative sizes and usage.
  */
 @InterfaceAudience.Private
-@JsonIgnoreProperties({"encodingCountsForTest"})
 public class LruBlockCache implements ResizableBlockCache, HeapSize {
 
   private static final Logger LOG = LoggerFactory.getLogger(LruBlockCache.class);
@@ -158,21 +152,23 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   private static final long DEFAULT_MAX_BLOCK_SIZE = 16L * 1024L * 1024L;
 
   /** Concurrent map (the cache) */
-  private final Map<BlockCacheKey, LruCachedBlock> map;
+  private transient final Map<BlockCacheKey, LruCachedBlock> map;
 
   /** Eviction lock (locked when eviction in process) */
-  private final ReentrantLock evictionLock = new ReentrantLock(true);
+  private transient final ReentrantLock evictionLock = new ReentrantLock(true);
+
   private final long maxBlockSize;
 
   /** Volatile boolean to track if we are in an eviction process or not */
   private volatile boolean evictionInProgress = false;
 
   /** Eviction thread */
-  private final EvictionThread evictionThread;
+  private transient final EvictionThread evictionThread;
 
   /** Statistics thread schedule pool (for heavy debugging, could remove) */
-  private final ScheduledExecutorService scheduleThreadPool = Executors.newScheduledThreadPool(1,
-    new ThreadFactoryBuilder().setNameFormat("LruBlockCacheStatsExecutor").setDaemon(true).build());
+  private transient final ScheduledExecutorService scheduleThreadPool =
+    Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
+      .setNameFormat("LruBlockCacheStatsExecutor").setDaemon(true).build());
 
   /** Current size of cache */
   private final AtomicLong size;
@@ -227,7 +223,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
    * external cache as L2.
    * Note: See org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache
    */
-  private BlockCache victimHandler = null;
+  private transient BlockCache victimHandler = null;
 
   /**
    * Default constructor.  Specify maximum size and expected average block
@@ -1170,10 +1166,10 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   }
 
   @Override
-  @JsonIgnore
   public BlockCache[] getBlockCaches() {
-    if (victimHandler != null)
-      return new BlockCache[] {this, this.victimHandler};
+    if (victimHandler != null) {
+      return new BlockCache[] { this, this.victimHandler };
+    }
     return null;
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
index 6caa975..09d5750 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
@@ -1,6 +1,4 @@
 /**
- * Copyright The Apache Software Foundation
- *
  * 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
@@ -28,14 +26,12 @@ import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.atomic.LongAdder;
-
 import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
+import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BucketEntry;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
-import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BucketEntry;
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 
 import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
@@ -44,19 +40,16 @@ import org.apache.hbase.thirdparty.com.google.common.primitives.Ints;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.map.LinkedMap;
 
 /**
- * This class is used to allocate a block with specified size and free the block
- * when evicting. It manages an array of buckets, each bucket is associated with
- * a size and caches elements up to this size. For a completely empty bucket, this
- * size could be re-specified dynamically.
- *
+ * This class is used to allocate a block with specified size and free the block when evicting. It
+ * manages an array of buckets, each bucket is associated with a size and caches elements up to this
+ * size. For a completely empty bucket, this size could be re-specified dynamically.
+ * <p/>
  * This class is not thread safe.
  */
 @InterfaceAudience.Private
-@JsonIgnoreProperties({"indexStatistics", "freeSize", "usedSize"})
 public final class BucketAllocator {
   private static final Logger LOG = LoggerFactory.getLogger(BucketAllocator.class);
 
-  @JsonIgnoreProperties({"completelyFree", "uninstantiated"})
   public final static class Bucket {
     private long baseOffset;
     private int itemAllocationSize, sizeIndex;
@@ -308,7 +301,7 @@ public final class BucketAllocator {
   private Bucket[] buckets;
   private BucketSizeInfo[] bucketSizeInfos;
   private final long totalSize;
-  private long usedSize = 0;
+  private transient long usedSize = 0;
 
   BucketAllocator(long availableSpace, int[] bucketSizes)
       throws BucketAllocatorException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 66ee0e4..5a21bad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -132,14 +132,14 @@ public class BucketCache implements BlockCache, HeapSize {
   final static int DEFAULT_WRITER_QUEUE_ITEMS = 64;
 
   // Store/read block data
-  final IOEngine ioEngine;
+  transient final IOEngine ioEngine;
 
   // Store the block in this map before writing it to cache
   @VisibleForTesting
-  final ConcurrentMap<BlockCacheKey, RAMQueueEntry> ramCache;
+  transient final ConcurrentMap<BlockCacheKey, RAMQueueEntry> ramCache;
   // In this map, store the block's meta data like offset, length
   @VisibleForTesting
-  ConcurrentMap<BlockCacheKey, BucketEntry> backingMap;
+  transient ConcurrentMap<BlockCacheKey, BucketEntry> backingMap;
 
   /**
    * Flag if the cache is enabled or not... We shut it off if there are IO
@@ -156,13 +156,13 @@ public class BucketCache implements BlockCache, HeapSize {
    * to the BucketCache.  It then updates the ramCache and backingMap accordingly.
    */
   @VisibleForTesting
-  final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues = new ArrayList<>();
+  transient final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues = new ArrayList<>();
   @VisibleForTesting
-  final WriterThread[] writerThreads;
+  transient final WriterThread[] writerThreads;
 
   /** Volatile boolean to track if free space is in process or not */
   private volatile boolean freeInProgress = false;
-  private final Lock freeSpaceLock = new ReentrantLock();
+  private transient final Lock freeSpaceLock = new ReentrantLock();
 
   private final LongAdder realCacheSize = new LongAdder();
   private final LongAdder heapSize = new LongAdder();
@@ -201,7 +201,7 @@ public class BucketCache implements BlockCache, HeapSize {
    * Key set of offsets in BucketCache is limited so soft reference is the best choice here.
    */
   @VisibleForTesting
-  final IdReadWriteLock<Long> offsetLock = new IdReadWriteLock<>(ReferenceType.SOFT);
+  transient final IdReadWriteLock<Long> offsetLock = new IdReadWriteLock<>(ReferenceType.SOFT);
 
   private final NavigableSet<BlockCacheKey> blocksByHFile =
       new ConcurrentSkipListSet<>(new Comparator<BlockCacheKey>() {
@@ -222,11 +222,12 @@ public class BucketCache implements BlockCache, HeapSize {
       });
 
   /** Statistics thread schedule pool (for heavy debugging, could remove) */
-  private final ScheduledExecutorService scheduleThreadPool = Executors.newScheduledThreadPool(1,
-    new ThreadFactoryBuilder().setNameFormat("BucketCacheStatsExecutor").setDaemon(true).build());
+  private transient final ScheduledExecutorService scheduleThreadPool =
+    Executors.newScheduledThreadPool(1,
+      new ThreadFactoryBuilder().setNameFormat("BucketCacheStatsExecutor").setDaemon(true).build());
 
   // Allocate or free space for the block
-  private BucketAllocator bucketAllocator;
+  private transient BucketAllocator bucketAllocator;
 
   /** Acceptable size of cache (no evictions if size < acceptable) */
   private float acceptableFactor;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 0207b03..3ab63dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -34,7 +34,6 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.atomic.LongAdder;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CallQueueTooBigException;
 import org.apache.hadoop.hbase.CellScanner;
@@ -55,6 +54,7 @@ import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
+import org.apache.hadoop.hbase.util.GsonUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -67,17 +67,17 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
 import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-
 /**
  * An RPC server that hosts protobuf described Services.
  *
@@ -195,7 +195,7 @@ public abstract class RpcServer implements RpcServerInterface,
   protected static final String TRACE_LOG_MAX_LENGTH = "hbase.ipc.trace.log.max.length";
   protected static final String KEY_WORD_TRUNCATED = " <TRUNCATED>";
 
-  protected static final ObjectMapper MAPPER = new ObjectMapper();
+  protected static final Gson GSON = GsonUtil.createGson().create();
 
   protected final int maxRequestSize;
   protected final int warnResponseTime;
@@ -517,7 +517,7 @@ public abstract class RpcServer implements RpcServerInterface,
         }
       }
     }
-    LOG.warn("(response" + tag + "): " + MAPPER.writeValueAsString(responseInfo));
+    LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
index 0fa638f..03f570d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
@@ -18,17 +18,17 @@
  */
 package org.apache.hadoop.hbase.monitoring;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.apache.hadoop.util.StringUtils;
-import org.apache.yetus.audience.InterfaceAudience;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
 
 @InterfaceAudience.Private
 class MonitoredTaskImpl implements MonitoredTask {
@@ -45,7 +45,7 @@ class MonitoredTaskImpl implements MonitoredTask {
   private boolean journalEnabled = false;
   private List<StatusJournalEntry> journal;
 
-  private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final Gson GSON = GsonUtil.createGson().create();
 
   public MonitoredTaskImpl() {
     startTime = System.currentTimeMillis();
@@ -213,7 +213,7 @@ class MonitoredTaskImpl implements MonitoredTask {
 
   @Override
   public String toJSON() throws IOException {
-    return MAPPER.writeValueAsString(toMap());
+    return GSON.toJson(toMap());
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
index ac5e773..031351d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
@@ -26,7 +26,6 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -39,9 +38,13 @@ import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.GsonUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
@@ -49,10 +52,6 @@ import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.PosixParser;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * WALPrettyPrinter prints the contents of a given WAL with a variety of
  * options affecting formatting and extent of content.
@@ -83,7 +82,7 @@ public class WALPrettyPrinter {
   // useful for programmatic capture of JSON output
   private PrintStream out;
   // for JSON encoding
-  private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final Gson GSON = GsonUtil.createGson().create();
   //allows for jumping straight to a given portion of the file
   private long position;
 
@@ -324,7 +323,7 @@ public class WALPrettyPrinter {
           else
             out.print(",");
           // encode and print JSON
-          out.print(MAPPER.writeValueAsString(txn));
+          out.print(GSON.toJson(txn));
         } else {
           // Pretty output, complete with indentation by atomic action
           out.println("Sequence=" + txn.get("sequence") + " "
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java
index 0274761..dbf54bb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestJSONMetricUtil.java
@@ -21,9 +21,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.JsonNode;
-import java.io.IOException;
 import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
 import java.util.Hashtable;
@@ -41,21 +38,21 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Category({MiscTests.class, SmallTests.class})
+@Category({ MiscTests.class, SmallTests.class })
 public class TestJSONMetricUtil {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestJSONMetricUtil.class);
+    HBaseClassTestRule.forClass(TestJSONMetricUtil.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestJSONMetricUtil.class);
 
   @Test
   public void testBuildHashtable() {
-    String[] keys = {"type", "name"};
+    String[] keys = { "type", "name" };
     String[] emptyKey = {};
-    String[] values = {"MemoryPool", "Par Eden Space"};
-    String[] values2 = {"MemoryPool", "Par Eden Space", "Test"};
+    String[] values = { "MemoryPool", "Par Eden Space" };
+    String[] values2 = { "MemoryPool", "Par Eden Space", "Test" };
     String[] emptyValue = {};
     Map<String, String> properties = JSONMetricUtil.buldKeyValueTable(keys, values);
     assertEquals(values[0], properties.get("type"));
@@ -68,27 +65,12 @@ public class TestJSONMetricUtil {
   }
 
   @Test
-  public void testSearchJson() throws JsonProcessingException, IOException {
-    String jsonString = "{\"test\":[{\"data1\":100,\"data2\":\"hello\",\"data3\": [1 , 2 , 3]}, "
-        + "{\"data4\":0}]}";
-    JsonNode  node = JSONMetricUtil.mappStringToJsonNode(jsonString);
-    JsonNode r1 = JSONMetricUtil.searchJson(node, "data1");
-    JsonNode r2 = JSONMetricUtil.searchJson(node, "data2");
-    JsonNode r3 = JSONMetricUtil.searchJson(node, "data3");
-    JsonNode r4 = JSONMetricUtil.searchJson(node, "data4");
-    assertEquals(100, r1.intValue());
-    assertEquals("hello", r2.textValue());
-    assertEquals(1, r3.get(0).intValue());
-    assertEquals(0, r4.intValue());
-  }
-
-  @Test
   public void testBuildObjectName() throws MalformedObjectNameException {
-    String[] keys = {"type", "name"};
-    String[] values = {"MemoryPool", "Par Eden Space"};
+    String[] keys = { "type", "name" };
+    String[] values = { "MemoryPool", "Par Eden Space" };
     Hashtable<String, String> properties = JSONMetricUtil.buldKeyValueTable(keys, values);
-    ObjectName testObject = JSONMetricUtil.buildObjectName(JSONMetricUtil.JAVA_LANG_DOMAIN,
-      properties);
+    ObjectName testObject =
+      JSONMetricUtil.buildObjectName(JSONMetricUtil.JAVA_LANG_DOMAIN, properties);
     assertEquals(JSONMetricUtil.JAVA_LANG_DOMAIN, testObject.getDomain());
     assertEquals(testObject.getKeyPropertyList(), properties);
   }
@@ -96,12 +78,12 @@ public class TestJSONMetricUtil {
   @Test
   public void testGetLastGCInfo() {
     List<GarbageCollectorMXBean> gcBeans = ManagementFactory.getGarbageCollectorMXBeans();
-    for(GarbageCollectorMXBean bean:gcBeans) {
+    for (GarbageCollectorMXBean bean : gcBeans) {
       ObjectName on = bean.getObjectName();
       Object value = JSONMetricUtil.getValueFromMBean(on, "LastGcInfo");
-      LOG.info("Collector Info: "+ value);
+      LOG.info("Collector Info: " + value);
       if (value != null && value instanceof CompositeData) {
-        CompositeData cds = (CompositeData)value;
+        CompositeData cds = (CompositeData) value;
         assertNotNull(cds.get("duration"));
       }
     }