Skip to content

Commit

Permalink
Resolve map(varchar, json) canonicalization bug
Browse files Browse the repository at this point in the history
The map function will not sort a json object by its keys, despite the
json_parse function sorting the same input.
If implemented, this will sort json objects.

Resolves #24207
  • Loading branch information
infvg committed Dec 12, 2024
1 parent c3e18d8 commit a096330
Show file tree
Hide file tree
Showing 2 changed files with 18 additions and 7 deletions.
18 changes: 16 additions & 2 deletions presto-main/src/main/java/com/facebook/presto/util/JsonUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package com.facebook.presto.util;

import com.facebook.airlift.json.JsonObjectMapperProvider;
import com.facebook.presto.common.block.Block;
import com.facebook.presto.common.block.BlockBuilder;
import com.facebook.presto.common.block.SingleRowBlockWriter;
Expand Down Expand Up @@ -41,6 +42,7 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.primitives.Shorts;
import com.google.common.primitives.SignedBytes;
import io.airlift.slice.DynamicSliceOutput;
import io.airlift.slice.Slice;
import io.airlift.slice.SliceOutput;
import io.airlift.slice.Slices;
Expand Down Expand Up @@ -86,6 +88,7 @@
import static com.fasterxml.jackson.core.JsonToken.FIELD_NAME;
import static com.fasterxml.jackson.core.JsonToken.START_ARRAY;
import static com.fasterxml.jackson.core.JsonToken.START_OBJECT;
import static com.fasterxml.jackson.databind.SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Verify.verify;
import static it.unimi.dsi.fastutil.HashCommon.arraySize;
Expand All @@ -106,6 +109,7 @@ public final class JsonUtil
// `OBJECT_MAPPER.writeValueAsString(parser.readValueAsTree());` preserves input order.
// Be aware. Using it arbitrarily can produce invalid json (ordered by key is required in Presto).
private static final ObjectMapper OBJECT_MAPPED_UNORDERED = new ObjectMapper(JSON_FACTORY);
private static final ObjectMapper OBJECT_MAPPED_SORTED = new JsonObjectMapperProvider().get().configure(ORDER_MAP_ENTRIES_BY_KEYS, true);

private static final int MAX_JSON_LENGTH_IN_ERROR_MESSAGE = 10_000;

Expand Down Expand Up @@ -956,8 +960,18 @@ static BlockBuilderAppender createBlockBuilderAppender(Type type)
return new VarcharBlockBuilderAppender(type);
case StandardTypes.JSON:
return (parser, blockBuilder, sqlFunctionProperties) -> {
String json = OBJECT_MAPPED_UNORDERED.writeValueAsString(parser.readValueAsTree());
JSON.writeSlice(blockBuilder, Slices.utf8Slice(json));
Slice slice = Slices.utf8Slice(OBJECT_MAPPED_UNORDERED.writeValueAsString(parser.readValueAsTree()));
try (JsonParser jsonParser = createJsonParser(JSON_FACTORY, slice)) {
SliceOutput dynamicSliceOutput = new DynamicSliceOutput(slice.length());
OBJECT_MAPPED_SORTED.writeValue((OutputStream) dynamicSliceOutput, OBJECT_MAPPED_SORTED.readValue(jsonParser, Object.class));
// nextToken() returns null if the input is parsed correctly,
// but will throw an exception if there are trailing characters.
jsonParser.nextToken();
JSON.writeSlice(blockBuilder, dynamicSliceOutput.slice());
}
catch (Exception e) {
throw new PrestoException(INVALID_FUNCTION_ARGUMENT, format("Cannot convert '%s' to JSON", slice.toStringUtf8()));
}
};
case StandardTypes.ARRAY:
return new ArrayBlockBuilderAppender(createBlockBuilderAppender(((ArrayType) type).getElementType()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -527,16 +527,13 @@ public void testJsonToMap()
.put("k8", "[null]")
.build());

// These two tests verifies that partial json cast preserves input order
// The second test should never happen in real life because valid json in presto requires natural key ordering.
// However, it is added to make sure that the order in the first test is not a coincidence.
assertFunction("CAST(JSON '{\"k1\": {\"1klmnopq\":1, \"2klmnopq\":2, \"3klmnopq\":3, \"4klmnopq\":4, \"5klmnopq\":5, \"6klmnopq\":6, \"7klmnopq\":7}}' AS MAP<VARCHAR, JSON>)",
mapType(VARCHAR, JSON),
ImmutableMap.of("k1", "{\"1klmnopq\":1,\"2klmnopq\":2,\"3klmnopq\":3,\"4klmnopq\":4,\"5klmnopq\":5,\"6klmnopq\":6,\"7klmnopq\":7}"));

assertFunction("CAST(unchecked_to_json('{\"k1\": {\"7klmnopq\":7, \"6klmnopq\":6, \"5klmnopq\":5, \"4klmnopq\":4, \"3klmnopq\":3, \"2klmnopq\":2, \"1klmnopq\":1}}') AS MAP<VARCHAR, JSON>)",
mapType(VARCHAR, JSON),
ImmutableMap.of("k1", "{\"7klmnopq\":7,\"6klmnopq\":6,\"5klmnopq\":5,\"4klmnopq\":4,\"3klmnopq\":3,\"2klmnopq\":2,\"1klmnopq\":1}"));

ImmutableMap.of("k1", "{\"1klmnopq\":1,\"2klmnopq\":2,\"3klmnopq\":3,\"4klmnopq\":4,\"5klmnopq\":5,\"6klmnopq\":6,\"7klmnopq\":7}"));
// nested array/map
assertFunction("CAST(JSON '{\"1\": [1, 2], \"2\": [3, null], \"3\": [], \"5\": [null, null], \"8\": null}' AS MAP<BIGINT, ARRAY<BIGINT>>)",
mapType(BIGINT, new ArrayType(BIGINT)),
Expand Down

0 comments on commit a096330

Please sign in to comment.