diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml index 573b4b7485f4..d33db894f75b 100644 --- a/hbase-client/pom.xml +++ b/hbase-client/pom.xml @@ -252,6 +252,10 @@ junit test + + org.apache.hbase.thirdparty + hbase-shaded-gson + 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 66b809b7c9b6..12a0a12a670b 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 map) throws IOException { + public static String writeMapAsString(Map 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 0d82fac3c258..119d29a184d0 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 @@ 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.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 TS_LIST = Arrays.asList(2L, 3L, 5L); private static TimestampsFilter TS_FILTER = new TimestampsFilter(TS_LIST); @@ -283,7 +285,9 @@ public void testOperationJSON() throws IOException { scan.addColumn(FAMILY, QUALIFIER); // get its JSON representation, and parse it String json = scan.toJSON(); - Map parsedJSON = mapper.readValue(json, HashMap.class); + Type typeOfHashMap = new TypeToken>() { + }.getType(); + Map 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 void testOperationJSON() throws IOException { 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 void testOperationJSON() throws IOException { 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 void testOperationJSON() throws IOException { 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 02d05aad4e71..a043b093d752 100644 --- a/hbase-common/pom.xml +++ b/hbase-common/pom.xml @@ -277,6 +277,10 @@ jackson-mapper-asl provided + + org.apache.hbase.thirdparty + hbase-shaded-gson + com.fasterxml.jackson.jaxrs jackson-jaxrs-json-provider 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 000000000000..d1c65ef78c19 --- /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. + *

+ * Will set some common configs for the builder. + */ + public static GsonBuilder createGson() { + return new GsonBuilder().setLongSerializationPolicy(LongSerializationPolicy.STRING) + .registerTypeAdapter(LongAdder.class, new TypeAdapter() { + + @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 165e7589e118..2dc4da487644 100644 --- a/hbase-resource-bundle/src/main/resources/supplemental-models.xml +++ b/hbase-resource-bundle/src/main/resources/supplemental-models.xml @@ -2112,6 +2112,25 @@ Copyright 2005 Sun Microsystems, Inc. and portions Copyright Apache Software Fou + + + com.google.errorprone + error_prone_annotations + + + Google + http://www.google.com + + + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + org.jamon diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index a07f42809e3c..dc47bbe128b4 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -492,6 +492,10 @@ org.apache.commons commons-math + + org.apache.hbase.thirdparty + hbase-shaded-gson + log4j log4j 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 0d7d99ab8c7f..872b874282ff 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 void doGet(HttpServletRequest request, HttpServletResponse response) thro 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 void doGet(HttpServletRequest request, HttpServletResponse response) thro } 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 void doGet(HttpServletRequest request, HttpServletResponse response) thro 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 bb80abee72e4..d044fa3e0304 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 8cbcae33743e..dfc279671446 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.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() { + + @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 String getFilename() { } /** - * @param filename - * @param blocks * @return A JSON String of filename and counts of blocks - * @throws JsonGenerationException - * @throws JsonMappingException - * @throws IOException */ public static String toJSON(final String filename, final NavigableSet blocks) - throws JsonGenerationException, JsonMappingException, IOException { + throws IOException { CachedBlockCountsPerFile counts = new CachedBlockCountsPerFile(filename); for (CachedBlock cb: blocks) { counts.count++; @@ -121,31 +135,21 @@ public static String toJSON(final String filename, final NavigableSetcbsf 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 bc 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 static boolean shouldReplaceExistingCacheBlock(BlockCache blockCache, * 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 static class CachedBlocksByFile { /** * Map by filename. use concurent utils because we want our Map and contained blocks sorted. */ - private NavigableMap> cachedBlockByFile = + private transient NavigableMap> cachedBlockByFile = new ConcurrentSkipListMap>(); 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 b854079642f2..5b4b88d67d6c 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 @@ * 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 map; + private transient final Map 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 @@ BlockCache getVictimHandler() { } @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 a28b990264a9..c8e726816253 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 @@ * 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 BucketSizeInfo roundUpToBucketSizeInfo(int blockSize) { 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 c5a1b21c055a..4923da6031f9 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 @@ -127,14 +127,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 ramCache; + transient final ConcurrentMap ramCache; // In this map, store the block's meta data like offset, length @VisibleForTesting - ConcurrentMap backingMap; + transient ConcurrentMap backingMap; /** * Flag if the cache is enabled or not... We shut it off if there are IO @@ -151,14 +151,14 @@ public class BucketCache implements BlockCache, HeapSize { * to the BucketCache. It then updates the ramCache and backingMap accordingly. */ @VisibleForTesting - final ArrayList> writerQueues = + transient final ArrayList> 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 UniqueIndexMap deserialiserMap = new UniqueIndexMap(); @@ -197,7 +197,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 blocksByHFile = new ConcurrentSkipListSet(new Comparator() { @@ -218,11 +218,12 @@ public int compare(BlockCacheKey a, BlockCacheKey b) { }); /** 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 dcadcaaf43bb..1b600bb1fbfe 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 @@ 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.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.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 @@ void logResponse(Message param, String methodName, String call, String tag, 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 7174b3f8413a..41705dbe9877 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.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 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 @@ public Map toMap() { @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 40889e0663f0..9a41506513dc 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 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. + *

+ * 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 int write(MBeanServer mBeanServer, ObjectName qry, String attribute, * @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 names = null; names = mBeanServer.queryNames(qry, null); - jg.writeArrayFieldStart("beans"); + writer.name("beans").beginArray(); Iterator it = names.iterator(); while (it.hasNext()) { ObjectName oname = it.next(); @@ -126,7 +146,9 @@ private static int write(final JsonGenerator jg, 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 @@ private static int write(final JsonGenerator jg, 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 @@ private static int write(final JsonGenerator jg, 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 @@ private static void writeAttribute(final JsonGenerator jg, 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 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 182c985847ac..c440c079eedf 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.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.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 void processFile(final Configuration conf, final Path p) 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 04df97d2bb86..2bf1ba7b37b2 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 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 @@ @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 void setStatus(String msg) { } }; - 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 @@ static Job doMapReduce(TestOptions opts, final Configuration conf) 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 @@ private static Path writeInputFile(final Configuration c, final TestOptions opts 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 @@ private void runTest(final Class cmd, TestOptions opts) throws I 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 397df5f07822..2a117bebb1ec 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 @@ 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 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()); } 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 beea3d61f2a8..11af8e8e7687 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 f254e4529db2..f0369b1eaa87 100644 --- a/hbase-testing-util/pom.xml +++ b/hbase-testing-util/pom.xml @@ -164,6 +164,11 @@ + + + org.codehaus.jackson + jackson-mapper-asl + org.apache.hbase hbase-server diff --git a/pom.xml b/pom.xml index 8ee04d4825ef..38246094a487 100644 --- a/pom.xml +++ b/pom.xml @@ -1307,6 +1307,7 @@ 2.11.6 1.46 1.0.1 + 3.0.0 2.1.1 3.1.1 @@ -1820,6 +1821,11 @@ jackson-databind ${jackson2.databind.version} + + org.apache.hbase.thirdparty + hbase-shaded-gson + ${hbase.shaded.gson.version} +