From 32e108c9fbb911c535fcd6c37777102992c2e661 Mon Sep 17 00:00:00 2001 From: Hazmi Date: Tue, 10 Dec 2024 16:50:16 +0300 Subject: [PATCH] Resolve map(varchar, json) canonicalization bug 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 --- .../com/facebook/presto/util/JsonUtil.java | 23 ++++-- .../operator/TestMapVarcharJsonOperator.java | 72 +++++++++++++++++++ 2 files changed, 91 insertions(+), 4 deletions(-) create mode 100644 presto-main/src/test/java/com/facebook/presto/operator/TestMapVarcharJsonOperator.java diff --git a/presto-main/src/main/java/com/facebook/presto/util/JsonUtil.java b/presto-main/src/main/java/com/facebook/presto/util/JsonUtil.java index 4b48dcbd8fff..64364220fdfa 100644 --- a/presto-main/src/main/java/com/facebook/presto/util/JsonUtil.java +++ b/presto-main/src/main/java/com/facebook/presto/util/JsonUtil.java @@ -38,7 +38,10 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.TreeNode; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.primitives.Shorts; import com.google.common.primitives.SignedBytes; import io.airlift.slice.Slice; @@ -53,6 +56,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -86,6 +90,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; @@ -102,10 +107,8 @@ public final class JsonUtil { public static final JsonFactory JSON_FACTORY = new JsonFactory().disable(CANONICALIZE_FIELD_NAMES); - // This object mapper is constructed without .configure(ORDER_MAP_ENTRIES_BY_KEYS, true) because - // `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_ORDERED = new ObjectMapper(JSON_FACTORY).configure(ORDER_MAP_ENTRIES_BY_KEYS, true); private static final int MAX_JSON_LENGTH_IN_ERROR_MESSAGE = 10_000; @@ -956,7 +959,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()); + TreeNode treeNode = parser.readValueAsTree(); + String json; + if (treeNode instanceof ObjectNode) { + Map recreatedChildren = new LinkedHashMap<>(); + ((ObjectNode) treeNode).fields().forEachRemaining(entry -> { + recreatedChildren.put(entry.getKey(), entry.getValue()); + }); + json = OBJECT_MAPPED_ORDERED.writeValueAsString(recreatedChildren); + } + else { + json = OBJECT_MAPPED_ORDERED.writeValueAsString(treeNode); + } JSON.writeSlice(blockBuilder, Slices.utf8Slice(json)); }; case StandardTypes.ARRAY: @@ -1165,6 +1179,7 @@ public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProp throws IOException { Slice result = currentTokenAsVarchar(parser); + String s = result.toStringUtf8(); if (result == null) { blockBuilder.appendNull(); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestMapVarcharJsonOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestMapVarcharJsonOperator.java new file mode 100644 index 000000000000..981491e82552 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestMapVarcharJsonOperator.java @@ -0,0 +1,72 @@ +/* + * Licensed 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 com.facebook.presto.operator; + +import com.facebook.presto.Session; +import com.facebook.presto.testing.LocalQueryRunner; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.testing.MaterializedRow; +import com.facebook.presto.testing.QueryRunner; +import com.google.common.collect.Iterables; +import org.intellij.lang.annotations.Language; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.HashMap; +import java.util.Map; + +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static java.lang.String.format; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class TestMapVarcharJsonOperator +{ + private QueryRunner queryRunner; + @BeforeClass + public void setUp() + { + Session session = testSessionBuilder().build(); + this.queryRunner = new LocalQueryRunner(session); + } + + @AfterClass(alwaysRun = true) + public void tearDown() + { + queryRunner.close(); + queryRunner = null; + } + + @Test + public void testFunction() + { + Map map = new HashMap<>(); + map.put("m", "[\"rn\",\"w\",\"a\"]"); + assertThatQueryReturnsValue("SELECT TRY(CAST(json_parse(c0) AS map(varchar, json))) from (values ('{\"m\": [\"rn\", \"w\", \"a\"]}')) t(c0)", map); + map.put("m", "{\"pl\":\"4\",\"rn\":\"w\"}"); + assertThatQueryReturnsValue("SELECT TRY(CAST(json_parse(c0) AS map(varchar, json))) from (values ('{\"m\": {\"rn\": \"w\", \"pl\": \"4\"}}')) t(c0)", map); + } + + private void assertThatQueryReturnsValue(@Language("SQL") String sql, Object expected) + { + MaterializedResult rows = queryRunner.execute(sql); + MaterializedRow materializedRow = Iterables.getOnlyElement(rows); + int fieldCount = materializedRow.getFieldCount(); + assertTrue(fieldCount == 1, format("Expected only one column, but got '%d'", fieldCount)); + Object value = materializedRow.getField(0); + assertEquals(value, expected); + assertTrue(Iterables.getOnlyElement(rows).getFieldCount() == 1); + } +}