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

[hbase] branch branch-1 updated: HBASE-23015 : Moving from Jackson2 to shaded Gson (#616)

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

busbey pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-1 by this push:
     new f77c14d  HBASE-23015 : Moving from Jackson2 to shaded Gson (#616)
f77c14d is described below

commit f77c14d18150f55ee892f8d24a5ee231c1ae7e20
Author: Viraj Jasani <vi...@gmail.com>
AuthorDate: Thu Sep 26 08:19:48 2019 +0530

    HBASE-23015 : Moving from Jackson2 to shaded Gson (#616)
    
    * Backport HBASE-20587
    * moving to shaded gson with jdk7 compatibility
    * Include jackson-mapper-asl for testing-util because of Hadoop
    * Update shaded jar check to allow hbase-thirdparty libs
    
    Signed-off-by: Sean Busbey <bu...@apache.org>
    Co-authored-by: Duo Zhang <zh...@apache.org>
---
 hbase-client/pom.xml                               |   4 +
 .../org/apache/hadoop/hbase/util/JsonMapper.java   |  11 +-
 .../apache/hadoop/hbase/client/TestOperation.java  |  20 ++-
 hbase-common/pom.xml                               |   4 +
 .../org/apache/hadoop/hbase/util/GsonUtil.java     |  60 +++++++
 .../src/main/resources/supplemental-models.xml     |  19 +++
 hbase-server/pom.xml                               |   4 +
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java      |   8 +-
 .../apache/hadoop/hbase/io/hfile/AgeSnapshot.java  |   6 +-
 .../hadoop/hbase/io/hfile/BlockCacheUtil.java      |  71 ++++----
 .../hadoop/hbase/io/hfile/LruBlockCache.java       |  23 ++-
 .../hbase/io/hfile/bucket/BucketAllocator.java     |   5 +-
 .../hadoop/hbase/io/hfile/bucket/BucketCache.java  |  21 +--
 .../org/apache/hadoop/hbase/ipc/RpcServer.java     |   7 +-
 .../hadoop/hbase/monitoring/MonitoredTaskImpl.java |   8 +-
 .../org/apache/hadoop/hbase/util/JSONBean.java     | 189 +++++++++++----------
 .../apache/hadoop/hbase/wal/WALPrettyPrinter.java  |   8 +-
 .../apache/hadoop/hbase/PerformanceEvaluation.java |  17 +-
 .../hadoop/hbase/TestPerformanceEvaluation.java    |   9 +-
 .../resources/ensure-jars-have-correct-contents.sh |   2 +
 hbase-testing-util/pom.xml                         |   5 +
 pom.xml                                            |   6 +
 22 files changed, 317 insertions(+), 190 deletions(-)

diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 573b4b7..d33db89 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -252,6 +252,10 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-gson</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 66b809b..12a0a12 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
@@ -18,12 +18,12 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.IOException;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
 
 /**
  * Utility class for converting objects to JSON
@@ -34,12 +34,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 0d82fac..119d29a 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;
 
@@ -60,6 +59,9 @@ import org.apache.hadoop.hbase.filter.WhileMatchFilter;
 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.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.reflect.TypeToken;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -75,7 +77,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);
@@ -283,7 +285,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"));
@@ -301,7 +305,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"));
@@ -319,7 +323,7 @@ public class TestOperation {
     put.add(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"));
@@ -333,14 +337,14 @@ public class TestOperation {
         Bytes.toStringBinary(QUALIFIER),
         kvMap.get("qualifier"));
     assertEquals("Value length incorrect in Put.toJSON()",
-        VALUE.length, kvMap.get("vlen"));
+      VALUE.length, ((Number) kvMap.get("vlen")).intValue());
 
     // produce a Delete operation
     Delete delete = new Delete(ROW);
     delete.deleteColumn(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 02d05aa..a043b09 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -278,6 +278,10 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-gson</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.fasterxml.jackson.jaxrs</groupId>
       <artifactId>jackson-jaxrs-json-provider</artifactId>
       <scope>provided</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..d1c65ef
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/GsonUtil.java
@@ -0,0 +1,60 @@
+/**
+ * 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 org.apache.hadoop.hbase.classification.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-resource-bundle/src/main/resources/supplemental-models.xml b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
index 165e758..2dc4da4 100644
--- a/hbase-resource-bundle/src/main/resources/supplemental-models.xml
+++ b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
@@ -2114,6 +2114,25 @@ Copyright 2005 Sun Microsystems, Inc. and portions Copyright Apache Software Fou
   </supplement>
   <supplement>
     <project>
+      <groupId>com.google.errorprone</groupId>
+      <artifactId>error_prone_annotations</artifactId>
+
+      <organization>
+        <name>Google</name>
+        <url>http://www.google.com</url>
+      </organization>
+      <licenses>
+        <license>
+          <!-- It has been incorrectly called Apache 2.0 in the original pom-->
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
       <groupId>org.jamon</groupId>
       <artifactId>jamon-runtime</artifactId>
       <name>Jamon runtime support classes</name>
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index a07f428..dc47bbe 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -493,6 +493,10 @@
       <artifactId>commons-math</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-gson</artifactId>
+    </dependency>
+    <dependency>
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
     </dependency>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
index 0d7d99a..872b874 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
@@ -163,7 +163,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) {
           response.setContentType("application/javascript; charset=utf8");
@@ -171,6 +170,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;
@@ -204,9 +204,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-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 bb80abe..d044fa3 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,18 +17,16 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import org.apache.hadoop.hbase.metrics.impl.FastLongHistogram;
 
 /**
  * Snapshot of block cache age in cache.
  * This object is preferred because we can control how it is serialized out when JSON'ing.
  */
-@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 8cbcae3..dfc2796 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
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-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 java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.NavigableMap;
@@ -35,6 +30,13 @@ import org.apache.hadoop.hbase.classification.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.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.
@@ -50,12 +52,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,15 +121,10 @@ 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 {
+      throws IOException {
     CachedBlockCountsPerFile counts = new CachedBlockCountsPerFile(filename);
     for (CachedBlock cb: blocks) {
       counts.count++;
@@ -121,31 +135,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(final 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(final BlockCache bc) throws IOException {
+    return GSON.toJson(bc);
   }
 
   /**
@@ -247,7 +251,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;
@@ -275,7 +278,7 @@ public class BlockCacheUtil {
     /**
      * Map by filename. use concurent utils because we want our Map and contained blocks sorted.
      */
-    private NavigableMap<String, NavigableSet<CachedBlock>> cachedBlockByFile =
+    private transient NavigableMap<String, NavigableSet<CachedBlock>> cachedBlockByFile =
       new ConcurrentSkipListMap<String, NavigableSet<CachedBlock>>();
     FastLongHistogram hist = new FastLongHistogram();
 
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 b854079..5b4b88d 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
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import java.lang.ref.WeakReference;
 import java.util.EnumMap;
 import java.util.Iterator;
@@ -96,7 +94,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * to the relative sizes and usage.
  */
 @InterfaceAudience.Private
-@JsonIgnoreProperties({"encodingCountsForTest"})
 public class LruBlockCache implements ResizableBlockCache, HeapSize {
 
   private static final Log LOG = LogFactory.getLog(LruBlockCache.class);
@@ -153,21 +150,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;
@@ -218,7 +217,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   private boolean forceInMemory;
 
   /** Where to send victims (blocks evicted/missing from the cache) */
-  private BlockCache victimHandler = null;
+  private transient BlockCache victimHandler = null;
 
   /**
    * Default constructor.  Specify maximum size and expected average block
@@ -1174,10 +1173,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 a28b990..c8e7268 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
@@ -20,7 +20,6 @@
 
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -52,11 +51,9 @@ import com.google.common.primitives.Ints;
  * This class is not thread safe.
  */
 @InterfaceAudience.Private
-@JsonIgnoreProperties({"indexStatistics", "freeSize", "usedSize"})
 public final class BucketAllocator {
   private static final Log LOG = LogFactory.getLog(BucketAllocator.class);
 
-  @JsonIgnoreProperties({"completelyFree", "uninstantiated"})
   public final static class Bucket {
     private long baseOffset;
     private int itemAllocationSize, sizeIndex;
@@ -308,7 +305,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 1e87a8e..daf96ef 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
@@ -129,14 +129,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
@@ -153,14 +153,14 @@ public class BucketCache implements BlockCache, HeapSize {
    * to the BucketCache.  It then updates the ramCache and backingMap accordingly.
    */
   @VisibleForTesting
-  final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues =
+  transient final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues =
       new ArrayList<BlockingQueue<RAMQueueEntry>>();
   @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 UniqueIndexMap<Integer> deserialiserMap = new UniqueIndexMap<Integer>();
 
@@ -199,7 +199,7 @@ public class BucketCache implements BlockCache, HeapSize {
    * The purpose of this is to avoid freeing the block which is being read.
    */
   @VisibleForTesting
-  final IdReadWriteLock offsetLock = new IdReadWriteLock();
+  transient final IdReadWriteLock offsetLock = new IdReadWriteLock();
 
   private final NavigableSet<BlockCacheKey> blocksByHFile =
       new ConcurrentSkipListSet<BlockCacheKey>(new Comparator<BlockCacheKey>() {
@@ -220,11 +220,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 dcadcaa..1b600bb 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
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.ipc;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.protobuf.BlockingService;
@@ -121,6 +120,7 @@ import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Counter;
+import org.apache.hadoop.hbase.util.GsonUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.io.BytesWritable;
@@ -138,6 +138,7 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
 import org.apache.htrace.TraceInfo;
 
 /**
@@ -279,7 +280,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   private static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds
   private static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
 
-  private static final ObjectMapper MAPPER = new ObjectMapper();
+  protected static final Gson GSON = GsonUtil.createGson().create();
 
   protected static final int DEFAULT_TRACE_LOG_MAX_LENGTH = 1000;
   protected static final String TRACE_LOG_MAX_LENGTH = "hbase.ipc.trace.log.max.length";
@@ -2524,7 +2525,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       responseInfo.put("multi.mutations", numMutations);
       responseInfo.put("multi.servicecalls", numServiceCalls);
     }
-    LOG.warn("(response" + tag + "): " + MAPPER.writeValueAsString(responseInfo));
+    LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
   }
 
   /** Stops the service.  No new calls will be handled after this is called. */
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 7174b3f..41705db 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,7 +18,6 @@
  */
 package org.apache.hadoop.hbase.monitoring;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 import java.io.IOException;
@@ -28,6 +27,9 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+
 @InterfaceAudience.Private
 class MonitoredTaskImpl implements MonitoredTask {
   private long startTime;
@@ -43,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();
@@ -210,7 +212,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/util/JSONBean.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
index 40889e0..9a41506 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
@@ -16,8 +16,6 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonGenerator;
 import java.io.BufferedWriter;
 import java.io.Closeable;
 import java.io.IOException;
@@ -48,54 +46,77 @@ import javax.management.openmbean.TabularData;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+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.
  */
 public class JSONBean {
   private static final Log LOG = LogFactory.getLog(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 IOException;
-    int write(final MBeanServer mBeanServer, ObjectName qry, String attribute,
-        final boolean description) throws IOException;
+
+    int write(final MBeanServer mBeanServer, final ObjectName qry, final String attribute,
+      final 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.createGenerator(writer);
-    jg.disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET);
-    jg.useDefaultPrettyPrinter();
-    jg.writeStartObject();
+    final 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 IOException {
-        jg.writeStringField(key, value);
+        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);
       }
     };
   }
@@ -108,14 +129,13 @@ public class JSONBean {
    * @return Return non-zero if failed to find bean. 0
    * @throws IOException
    */
-  private static int write(final JsonGenerator jg,
-      final MBeanServer mBeanServer, ObjectName qry, String attribute,
-      final boolean description)
-  throws IOException {
-    LOG.trace("Listing beans for "+qry);
+  private static int write(final JsonWriter writer, final MBeanServer mBeanServer,
+      final ObjectName qry, final String attribute, final boolean description) throws IOException {
+
+    LOG.trace("Listing beans for " + qry);
     Set<ObjectName> names = null;
     names = mBeanServer.queryNames(qry, null);
-    jg.writeArrayFieldStart("beans");
+    writer.name("beans").beginArray();
     Iterator<ObjectName> it = names.iterator();
     while (it.hasNext()) {
       ObjectName oname = it.next();
@@ -126,7 +146,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)) {
@@ -138,16 +160,16 @@ 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.
@@ -188,39 +210,38 @@ public class JSONBean {
         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(final JsonWriter writer, final MBeanServer mBeanServer,
+      final ObjectName oname, final boolean description, final MBeanAttributeInfo attr)
+      throws IOException {
     if (!attr.isReadable()) {
       return;
     }
@@ -278,71 +299,67 @@ public class JSONBean {
       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 {
+  private static void writeAttribute(JsonWriter writer, String attName, String descriptionStr,
+    Object value) throws IOException {
     boolean description = false;
     if (descriptionStr != null && descriptionStr.length() > 0 && !attName.equals(descriptionStr)) {
-      description = true;
-      jg.writeFieldName(attName);
-      jg.writeStartObject();
-      jg.writeFieldName("description");
-      jg.writeString(descriptionStr);
-      jg.writeFieldName("value");
-      writeObject(jg, description, value);
-      jg.writeEndObject();
+      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(final JsonWriter writer, final 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();
+        writer.endArray();
       } else if(value instanceof Number) {
         Number n = (Number)value;
         double doubleValue = n.doubleValue();
         if (Double.isNaN(doubleValue) || Double.isInfinite(doubleValue)) {
-          jg.writeString(n.toString());
+          writer.value(n);
         } else {
-          jg.writeNumber(n.toString());
+          writer.value(n.toString());
         }
       } else if(value instanceof Boolean) {
         Boolean b = (Boolean)value;
-        jg.writeBoolean(b);
+        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();
+        writer.endObject();
       } else if(value instanceof TabularData) {
         TabularData tds = (TabularData)value;
-        jg.writeStartArray();
-        for(Object entry : tds.values()) {
-          writeObject(jg, description, entry);
+        writer.beginArray();
+        for (Object entry : tds.values()) {
+          writeObject(writer, entry);
         }
-        jg.writeEndArray();
+        writer.endArray();
       } else {
-        jg.writeString(value.toString());
+        writer.value(value.toString());
       }
     }
   }
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 182c985..c440c07 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
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.wal;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
@@ -36,6 +35,7 @@ import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.PosixParser;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.GsonUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -50,6 +50,8 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
 // imports for things that haven't moved yet.
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+
 
 /**
  * WALPrettyPrinter prints the contents of a given WAL with a variety of
@@ -79,7 +81,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();
 
   private long position;
 
@@ -313,7 +315,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/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 04df97d..2bf1ba7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase;
 
-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;
@@ -102,6 +100,7 @@ import org.apache.hadoop.util.ToolRunner;
 import com.yammer.metrics.core.Histogram;
 import com.yammer.metrics.stats.UniformSample;
 
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
 import org.apache.htrace.Sampler;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
@@ -127,10 +126,7 @@ import org.apache.htrace.impl.ProbabilitySampler;
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 public class PerformanceEvaluation extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(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";
@@ -291,8 +287,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);
 
@@ -506,7 +501,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
 
     TableMapReduceUtil.initCredentials(job);
 
@@ -540,7 +535,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
           TestOptions next = new TestOptions(opts);
           next.startRow = (j * perClientRows) + (i * (perClientRows/10));
           next.perClientRunRows = perClientRows / 10;
-          String s = MAPPER.writeValueAsString(next);
+          String s = GSON.toJson(next);
           LOG.info("maptask input=" + s);
           int hash = h.hash(Bytes.toBytes(s));
           m.put(hash, s);
@@ -1948,7 +1943,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
       InterruptedException, ClassNotFoundException {
     // 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));
     try(Connection conn = ConnectionFactory.createConnection(getConf());
         Admin admin = conn.getAdmin()) {
       checkTable(admin, opts);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
index 397df5f..2a117be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
@@ -19,13 +19,14 @@ package org.apache.hadoop.hbase;
 
 import static org.junit.Assert.*;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.IOException;
 import java.util.NoSuchElementException;
 import java.util.Queue;
 import java.util.LinkedList;
 
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -37,10 +38,10 @@ public class TestPerformanceEvaluation {
     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());
   }
 
diff --git a/hbase-shaded/hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh b/hbase-shaded/hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
index beea3d6..11af8e8 100644
--- a/hbase-shaded/hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
+++ b/hbase-shaded/hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
@@ -28,6 +28,8 @@ allowed_expr="(^org/$|^org/apache/$"
 #   * classes in packages that start with org.apache.hadoop, which by
 #     convention should be in a path that looks like org/apache/hadoop
 allowed_expr+="|^org/apache/hadoop/"
+#   * classes in packages that start with org.apache.hbase
+allowed_expr+="|^org/apache/hbase/"
 #   * whatever in the "META-INF" directory
 allowed_expr+="|^META-INF/"
 #   * the folding tables from jcodings
diff --git a/hbase-testing-util/pom.xml b/hbase-testing-util/pom.xml
index f254e45..f0369b1 100644
--- a/hbase-testing-util/pom.xml
+++ b/hbase-testing-util/pom.xml
@@ -164,6 +164,11 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <!-- Adding jackson-mapper-asl dependency as Hadoop Minicluster requires it -->
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-mapper-asl</artifactId>
+        </dependency>
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-server</artifactId>
diff --git a/pom.xml b/pom.xml
index 8ee04d4..3824609 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1307,6 +1307,7 @@
     <spy.version>2.11.6</spy.version>
     <bouncycastle.version>1.46</bouncycastle.version>
     <kerby.version>1.0.1</kerby.version>
+    <hbase.shaded.gson.version>3.0.0</hbase.shaded.gson.version>
     <!-- Plugin Dependencies -->
     <error-prone.version>2.1.1</error-prone.version>
     <maven.assembly.version>3.1.1</maven.assembly.version>
@@ -1821,6 +1822,11 @@
         <version>${jackson2.databind.version}</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.hbase.thirdparty</groupId>
+        <artifactId>hbase-shaded-gson</artifactId>
+        <version>${hbase.shaded.gson.version}</version>
+      </dependency>
+      <dependency>
         <!--If this is not in the runtime lib, we get odd
       "2009-02-27 11:38:39.504::WARN:  failed jsp
        java.lang.NoSuchFieldError: IS_SECURITY_ENABLED"