Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.JsonDeserializer;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import com.fasterxml.jackson.databind.json.JsonMapper;
import com.google.errorprone.annotations.Immutable;

import java.io.IOException;
Expand Down Expand Up @@ -165,7 +165,7 @@ public int hashCode()
public static class ClientTypeSignatureParameterDeserializer
extends JsonDeserializer<ClientTypeSignatureParameter>
{
private static final ObjectMapper MAPPER = TrinoJsonCodec.OBJECT_MAPPER_SUPPLIER.get();
private static final JsonMapper MAPPER = TrinoJsonCodec.JSON_MAPPER_SUPPLIER.get();

@Override
public ClientTypeSignatureParameter deserialize(JsonParser jp, DeserializationContext ctxt)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@

import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.json.JsonMapper;
import com.google.common.collect.ImmutableList;

import java.io.IOException;
Expand Down Expand Up @@ -467,13 +467,13 @@ public Object decode(JsonParser parser)
private static class ObjectDecoder
implements TypeDecoder
{
private final ObjectMapper objectMapper = new ObjectMapper();
private final JsonMapper jsonMapper = new JsonMapper();

@Override
public Object decode(JsonParser parser)
throws IOException
{
return objectMapper.readValue(parser, Object.class);
return jsonMapper.readValue(parser, Object.class);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.json.JsonMapper;
import com.google.common.base.VerifyException;
import com.google.common.collect.AbstractIterator;
import com.google.common.io.Closer;
Expand All @@ -41,7 +41,7 @@

public final class JsonIterators
{
private static final JsonFactory JSON_FACTORY = createJsonFactory();
static final JsonMapper JSON_MAPPER = new JsonMapper(createJsonFactory());

private JsonIterators() {}

Expand Down Expand Up @@ -81,7 +81,7 @@ public JsonIterator(JsonParser parser, TypeDecoder[] decoders)
public JsonIterator(InputStream stream, TypeDecoder[] decoders)
throws IOException
{
this(JSON_FACTORY.createParser(requireNonNull(stream, "stream is null")), decoders);
this(JSON_MAPPER.createParser(requireNonNull(stream, "stream is null")), decoders);
closer.register(stream);
}

Expand Down Expand Up @@ -154,7 +154,6 @@ public static CloseableIterator<List<Object>> forInputStream(InputStream stream,
static JsonFactory createJsonFactory()
{
return new JsonFactory()
.setCodec(new ObjectMapper())
.enable(USE_FAST_DOUBLE_PARSER)
.enable(USE_FAST_BIG_NUMBER_PARSER)
.disable(AUTO_CLOSE_SOURCE); // We want to close source explicitly
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.JavaType;
import com.fasterxml.jackson.databind.MapperFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.json.JsonMapper;
import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;

Expand All @@ -38,8 +37,8 @@

public class TrinoJsonCodec<T>
{
// copy of https://github.com/airlift/airlift/blob/master/json/src/main/java/io/airlift/json/ObjectMapperProvider.java
static final Supplier<ObjectMapper> OBJECT_MAPPER_SUPPLIER = () -> {
// copy of https://github.com/airlift/airlift/blob/master/json/src/main/java/io/airlift/json/JsonMapperProvider.java
static final Supplier<JsonMapper> JSON_MAPPER_SUPPLIER = () -> {
JsonFactory jsonFactory = JsonFactory.builder()
/*
* When multiple threads deserialize JSON responses concurrently,
Expand Down Expand Up @@ -82,14 +81,14 @@ public class TrinoJsonCodec<T>

public static <T> TrinoJsonCodec<T> jsonCodec(Class<T> type)
{
return new TrinoJsonCodec<>(OBJECT_MAPPER_SUPPLIER.get(), type);
return new TrinoJsonCodec<>(JSON_MAPPER_SUPPLIER.get(), type);
}

private final ObjectMapper mapper;
private final JsonMapper mapper;
private final Type type;
private final JavaType javaType;

private TrinoJsonCodec(ObjectMapper mapper, Type type)
private TrinoJsonCodec(JsonMapper mapper, Type type)
{
this.mapper = requireNonNull(mapper, "mapper is null");
this.type = requireNonNull(type, "type is null");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
import com.google.common.collect.ImmutableList;
import io.airlift.json.JsonCodec;
import io.airlift.json.JsonCodecFactory;
import io.airlift.json.ObjectMapperProvider;
import io.trino.spi.type.StandardTypes;
import org.junit.jupiter.api.Test;

Expand All @@ -29,9 +28,7 @@ public class TestClientTypeSignature
public static final JsonCodec<ClientTypeSignature> CLIENT_TYPE_SIGNATURE_CODEC;

static {
ObjectMapperProvider provider = new ObjectMapperProvider();
JsonCodecFactory codecFactory = new JsonCodecFactory(provider);
CLIENT_TYPE_SIGNATURE_CODEC = codecFactory.jsonCodec(ClientTypeSignature.class);
CLIENT_TYPE_SIGNATURE_CODEC = new JsonCodecFactory().jsonCodec(ClientTypeSignature.class);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.IntStream;

import static io.trino.client.JsonIterators.createJsonFactory;
import static io.trino.client.JsonIterators.JSON_MAPPER;
import static io.trino.client.spooling.Segment.inlined;
import static io.trino.client.spooling.Segment.spooled;
import static java.nio.charset.StandardCharsets.UTF_8;
Expand Down Expand Up @@ -69,7 +69,7 @@ public void testTypedJsonMaterialization()
public void testJsonNodeMaterialization()
throws Exception
{
try (ResultRowsDecoder decoder = new ResultRowsDecoder(); JsonParser parser = createJsonFactory().createParser("[[2137], [1337]]")) {
try (ResultRowsDecoder decoder = new ResultRowsDecoder(); JsonParser parser = JSON_MAPPER.createParser("[[2137], [1337]]")) {
assertThat(eagerlyMaterialize(decoder.toRows(fromQueryData(new JsonQueryData(parser.readValueAsTree())))))
.containsExactly(ImmutableList.of(2137), ImmutableList.of(1337));
}
Expand All @@ -90,7 +90,7 @@ public void testEagerInlineJsonNodeScanningMaterialization()
throws Exception
{
CountingInputStream stream = new CountingInputStream(new ByteArrayInputStream("[[2137], [1337]]".getBytes(UTF_8)));
try (ResultRowsDecoder decoder = new ResultRowsDecoder(); JsonParser parser = createJsonFactory().createParser(stream)) {
try (ResultRowsDecoder decoder = new ResultRowsDecoder(); JsonParser parser = JSON_MAPPER.createParser(stream)) {
Iterator<List<Object>> iterator = decoder.toRows(fromQueryData(new JsonQueryData(parser.readValueAsTree()))).iterator();
assertThat(stream.getCount()).isEqualTo(16);
iterator.next();
Expand All @@ -103,7 +103,7 @@ public void testLazyInlineJsonNodeScanningMaterialization()
throws Exception
{
CountingInputStream stream = new CountingInputStream(new ByteArrayInputStream("[[2137], [1337]]".getBytes(UTF_8)));
try (ResultRowsDecoder decoder = new ResultRowsDecoder(); JsonParser parser = createJsonFactory().createParser(stream)) {
try (ResultRowsDecoder decoder = new ResultRowsDecoder(); JsonParser parser = JSON_MAPPER.createParser(stream)) {
Iterator<List<Object>> iterator = decoder.toRows(fromQueryData(new JsonQueryData(parser.readValueAsTree()))).iterator();
assertThat(stream.getCount()).isEqualTo(16);
iterator.next();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
import com.google.common.collect.ImmutableSet;
import io.airlift.json.JsonCodec;
import io.airlift.json.JsonCodecFactory;
import io.airlift.json.ObjectMapperProvider;
import io.airlift.json.JsonMapperProvider;
import io.trino.client.ClientTypeSignature;
import io.trino.client.Column;
import io.trino.client.QueryData;
Expand Down Expand Up @@ -55,7 +55,7 @@
@Execution(SAME_THREAD)
public class TestProgressMonitor
{
private static final JsonCodec<QueryResults> QUERY_RESULTS_CODEC = new JsonCodecFactory(new ObjectMapperProvider()
private static final JsonCodec<QueryResults> QUERY_RESULTS_CODEC = new JsonCodecFactory(new JsonMapperProvider()
.withModules(ImmutableSet.of(new ServerQueryDataJacksonModule())))
.jsonCodec(QueryResults.class);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,8 @@
*/
package io.trino.operator.scalar;

import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.json.JsonMapper;
import com.google.common.collect.ImmutableList;
import io.airlift.slice.DynamicSliceOutput;
import io.airlift.slice.Slice;
Expand Down Expand Up @@ -50,7 +50,7 @@ public class ArrayToJsonCast

private static final MethodHandle METHOD_HANDLE = methodHandle(ArrayToJsonCast.class, "toJson", JsonGeneratorWriter.class, Block.class);

private static final JsonFactory JSON_FACTORY = createJsonFactory();
private static final JsonMapper JSON_MAPPER = new JsonMapper(createJsonFactory());

private ArrayToJsonCast()
{
Expand Down Expand Up @@ -82,7 +82,7 @@ public static Slice toJson(JsonGeneratorWriter writer, Block block)
{
try {
SliceOutput output = new DynamicSliceOutput(40);
try (JsonGenerator jsonGenerator = createJsonGenerator(JSON_FACTORY, output)) {
try (JsonGenerator jsonGenerator = createJsonGenerator(JSON_MAPPER, output)) {
jsonGenerator.writeStartArray();
for (int i = 0; i < block.getPositionCount(); i++) {
writer.writeJsonValue(jsonGenerator, block, i);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,12 @@
*/
package io.trino.operator.scalar;

import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParseException;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.core.io.SerializedString;
import com.fasterxml.jackson.databind.json.JsonMapper;
import com.google.common.collect.ImmutableList;
import io.airlift.slice.DynamicSliceOutput;
import io.airlift.slice.Slice;
Expand Down Expand Up @@ -118,16 +118,16 @@ public final class JsonExtract
{
private static final int ESTIMATED_JSON_OUTPUT_SIZE = 512;

private static final JsonFactory JSON_FACTORY = jsonFactoryBuilder()
private static final JsonMapper JSON_MAPPER = new JsonMapper(jsonFactoryBuilder()
.disable(CANONICALIZE_FIELD_NAMES)
.build();
.build());

private JsonExtract() {}

public static <T> T extract(Slice jsonInput, JsonExtractor<T> jsonExtractor)
{
requireNonNull(jsonInput, "jsonInput is null");
try (JsonParser jsonParser = createJsonParser(JSON_FACTORY, jsonInput)) {
try (JsonParser jsonParser = createJsonParser(JSON_MAPPER, jsonInput)) {
return extract(jsonParser, jsonExtractor);
}
catch (IOException e) {
Expand Down Expand Up @@ -299,7 +299,7 @@ public Slice extract(JsonParser jsonParser)
}

DynamicSliceOutput dynamicSliceOutput = new DynamicSliceOutput(ESTIMATED_JSON_OUTPUT_SIZE);
try (JsonGenerator jsonGenerator = createJsonGenerator(JSON_FACTORY, dynamicSliceOutput)) {
try (JsonGenerator jsonGenerator = createJsonGenerator(JSON_MAPPER, dynamicSliceOutput)) {
jsonGenerator.copyCurrentStructure(jsonParser);
}
return dynamicSliceOutput.slice();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,10 +13,10 @@
*/
package io.trino.operator.scalar;

import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.databind.MappingJsonFactory;
import com.fasterxml.jackson.databind.json.JsonMapper;
import com.google.common.primitives.Doubles;
import io.airlift.slice.Slice;
import io.trino.plugin.base.util.JsonTypeUtil;
Expand Down Expand Up @@ -54,12 +54,13 @@

public final class JsonFunctions
{
private static final JsonFactory JSON_FACTORY = jsonFactoryBuilder()
private static final JsonMapper JSON_MAPPER = new JsonMapper(jsonFactoryBuilder()
.disable(CANONICALIZE_FIELD_NAMES)
.build();
.build());

private static final JsonFactory MAPPING_JSON_FACTORY = new MappingJsonFactory()
.disable(CANONICALIZE_FIELD_NAMES);
private static final JsonMapper MAPPING_JSON_MAPPER = new JsonMapper(MappingJsonFactory.builder()
.disable(CANONICALIZE_FIELD_NAMES)
.build());

private JsonFunctions() {}

Expand Down Expand Up @@ -91,7 +92,7 @@ public static boolean varcharIsJsonScalar(@SqlType("varchar(x)") Slice json)
@SqlType(StandardTypes.BOOLEAN)
public static boolean isJsonScalar(@SqlType(StandardTypes.JSON) Slice json)
{
try (JsonParser parser = createJsonParser(JSON_FACTORY, json)) {
try (JsonParser parser = createJsonParser(JSON_MAPPER, json)) {
JsonToken nextToken = parser.nextToken();
if (nextToken == null) {
throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Invalid JSON value: " + truncateIfNecessaryForErrorMessage(json));
Expand Down Expand Up @@ -146,7 +147,7 @@ public static Long varcharJsonArrayLength(@SqlType("varchar(x)") Slice json)
@SqlType(StandardTypes.BIGINT)
public static Long jsonArrayLength(@SqlType(StandardTypes.JSON) Slice json)
{
try (JsonParser parser = createJsonParser(JSON_FACTORY, json)) {
try (JsonParser parser = createJsonParser(JSON_MAPPER, json)) {
if (parser.nextToken() != START_ARRAY) {
return null;
}
Expand Down Expand Up @@ -183,7 +184,7 @@ public static Boolean varcharJsonArrayContains(@SqlType("varchar(x)") Slice json
@SqlType(StandardTypes.BOOLEAN)
public static Boolean jsonArrayContains(@SqlType(StandardTypes.JSON) Slice json, @SqlType(StandardTypes.BOOLEAN) boolean value)
{
try (JsonParser parser = createJsonParser(JSON_FACTORY, json)) {
try (JsonParser parser = createJsonParser(JSON_MAPPER, json)) {
if (parser.nextToken() != START_ARRAY) {
return null;
}
Expand Down Expand Up @@ -223,7 +224,7 @@ public static Boolean varcharJsonArrayContains(@SqlType("varchar(x)") Slice json
@SqlType(StandardTypes.BOOLEAN)
public static Boolean jsonArrayContains(@SqlType(StandardTypes.JSON) Slice json, @SqlType(StandardTypes.BIGINT) long value)
{
try (JsonParser parser = createJsonParser(JSON_FACTORY, json)) {
try (JsonParser parser = createJsonParser(JSON_MAPPER, json)) {
if (parser.nextToken() != START_ARRAY) {
return null;
}
Expand Down Expand Up @@ -268,7 +269,7 @@ public static Boolean jsonArrayContains(@SqlType(StandardTypes.JSON) Slice json,
return false;
}

try (JsonParser parser = createJsonParser(JSON_FACTORY, json)) {
try (JsonParser parser = createJsonParser(JSON_MAPPER, json)) {
if (parser.nextToken() != START_ARRAY) {
return null;
}
Expand Down Expand Up @@ -312,7 +313,7 @@ public static Boolean jsonArrayContains(@SqlType(StandardTypes.JSON) Slice json,
{
String valueString = value.toStringUtf8();

try (JsonParser parser = createJsonParser(JSON_FACTORY, json)) {
try (JsonParser parser = createJsonParser(JSON_MAPPER, json)) {
if (parser.nextToken() != START_ARRAY) {
return null;
}
Expand Down Expand Up @@ -356,7 +357,7 @@ public static Slice jsonArrayGet(@SqlType(StandardTypes.JSON) Slice json, @SqlTy
return null;
}

try (JsonParser parser = createJsonParser(MAPPING_JSON_FACTORY, json)) {
try (JsonParser parser = createJsonParser(MAPPING_JSON_MAPPER, json)) {
if (parser.nextToken() != START_ARRAY) {
return null;
}
Expand Down
Loading