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 prestodb#24207
  • Loading branch information
infvg committed Dec 10, 2024
1 parent c3e18d8 commit 8a3f3db
Show file tree
Hide file tree
Showing 2 changed files with 79 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import com.facebook.presto.common.type.TypeSignature;
import com.facebook.presto.common.type.TypeSignatureParameter;
import com.facebook.presto.common.type.UnknownType;
import com.facebook.presto.operator.scalar.JsonFunctions;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.type.BigintOperators;
import com.facebook.presto.type.BooleanOperators;
Expand All @@ -38,7 +39,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;
Expand All @@ -53,6 +57,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;
Expand Down Expand Up @@ -86,6 +91,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 Down Expand Up @@ -957,7 +963,7 @@ static BlockBuilderAppender createBlockBuilderAppender(Type type)
case StandardTypes.JSON:
return (parser, blockBuilder, sqlFunctionProperties) -> {
String json = OBJECT_MAPPED_UNORDERED.writeValueAsString(parser.readValueAsTree());
JSON.writeSlice(blockBuilder, Slices.utf8Slice(json));
JSON.writeSlice(blockBuilder, JsonFunctions.jsonParse(Slices.utf8Slice(json)));
};
case StandardTypes.ARRAY:
return new ArrayBlockBuilderAppender(createBlockBuilderAppender(((ArrayType) type).getElementType()));
Expand Down
Original file line number Diff line number Diff line change
@@ -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<String, String> 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);
}
}

0 comments on commit 8a3f3db

Please sign in to comment.