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 <busbey@apache.org>
Co-authored-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
Viraj Jasani 2019-09-26 08:19:48 +05:30 committed by Sean Busbey
parent 771e184376
commit f77c14d181
22 changed files with 318 additions and 191 deletions

View File

@ -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>

View File

@ -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);
}
}

View File

@ -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"));

View File

@ -277,6 +277,10 @@
<artifactId>jackson-mapper-asl</artifactId>
<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>

View File

@ -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;
}
});
}
}

View File

@ -2112,6 +2112,25 @@ Copyright 2005 Sun Microsystems, Inc. and portions Copyright Apache Software Fou
</licenses>
</project>
</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>

View File

@ -492,6 +492,10 @@
<groupId>org.apache.commons</groupId>
<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>

View File

@ -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();

View File

@ -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;

View File

@ -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();

View File

@ -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;
}
}

View File

@ -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 {

View File

@ -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;

View File

@ -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. */

View File

@ -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

View File

@ -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();
return new Writer() {
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 {
jg.flush();
writer.flush();
}
@Override
public void close() throws IOException {
jg.close();
// do nothing
}
});
jsonWriter.setIndent(" ");
jsonWriter.beginObject();
return new Writer() {
@Override
public void flush() throws IOException {
jsonWriter.flush();
}
@Override
public void close() throws IOException {
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());
}
}
}

View File

@ -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") + " "

View File

@ -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);

View File

@ -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());
}

View File

@ -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

View File

@ -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>

View File

@ -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>
@ -1820,6 +1821,11 @@
<artifactId>jackson-databind</artifactId>
<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