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 {
return writeObjectAsString(map);
}
public static String writeObjectAsString(Object object) throws IOException {
return MAPPER.writeValueAsString(object);
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,7 +204,9 @@ 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(");");
}

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,13 +52,30 @@ 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
* @return The block content as String.
@ -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)
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,
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,
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 {
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)) {
@ -188,38 +210,37 @@ 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)
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 {
private static void writeObject(final JsonWriter writer, final Object value) throws IOException {
if (value == null) {
jg.writeNull();
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();
writer.beginObject();
for (String key : keys) {
writeAttribute(jg, key, null, cds.get(key));
writeAttribute(writer, key, null, cds.get(key));
}
jg.writeEndObject();
writer.endObject();
} else if(value instanceof TabularData) {
TabularData tds = (TabularData)value;
jg.writeStartArray();
writer.beginArray();
for (Object entry : tds.values()) {
writeObject(jg, description, entry);
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