From 73c461790e4a74f475e4795b21c78b00c2994309 Mon Sep 17 00:00:00 2001 From: kasiafi <30203062+kasiafi@users.noreply.github.com> Date: Fri, 12 Nov 2021 18:07:00 +0100 Subject: [PATCH] Analyze JSON_EXISTS, JSON_VALUE and JSON_QUERY functions --- .../io/trino/json/JsonInputErrorNode.java | 169 ++++++ .../trino/metadata/SystemFunctionBundle.java | 4 + .../java/io/trino/metadata/TypeRegistry.java | 2 + .../scalar/json/JsonExistsFunction.java | 103 ++++ .../scalar/json/JsonInputConversionError.java | 32 ++ .../scalar/json/JsonInputFunctions.java | 114 ++++ .../json/JsonOutputConversionError.java | 32 ++ .../scalar/json/JsonOutputFunctions.java | 168 ++++++ .../scalar/json/JsonQueryFunction.java | 110 ++++ .../scalar/json/JsonValueFunction.java | 116 ++++ .../io/trino/server/ServerMainModule.java | 26 + .../sql/analyzer/AggregationAnalyzer.java | 34 ++ .../java/io/trino/sql/analyzer/Analysis.java | 36 ++ .../sql/analyzer/ExpressionAnalyzer.java | 430 +++++++++++++- .../trino/sql/analyzer/JsonPathAnalyzer.java | 535 ++++++++++++++++++ .../io/trino/sql/planner/TranslationMap.java | 21 + .../io/trino/testing/LocalQueryRunner.java | 23 +- .../main/java/io/trino/type/Json2016Type.java | 83 +++ .../java/io/trino/type/JsonPath2016Type.java | 86 +++ .../io/trino/sql/analyzer/TestAnalyzer.java | 343 +++++++++++ .../main/antlr4/io/trino/jsonpath/JsonPath.g4 | 200 +++++++ .../io/trino/sql/jsonpath/PathNodeRef.java | 68 +++ .../io/trino/sql/jsonpath/PathParser.java | 167 ++++++ .../trino/sql/jsonpath/PathTreeBuilder.java | 421 ++++++++++++++ .../io/trino/sql/jsonpath/tree/AbsMethod.java | 29 + .../io/trino/sql/jsonpath/tree/Accessor.java | 38 ++ .../sql/jsonpath/tree/ArithmeticBinary.java | 61 ++ .../sql/jsonpath/tree/ArithmeticUnary.java | 51 ++ .../sql/jsonpath/tree/ArrayAccessor.java | 70 +++ .../sql/jsonpath/tree/CeilingMethod.java | 29 + .../jsonpath/tree/ComparisonPredicate.java | 62 ++ .../jsonpath/tree/ConjunctionPredicate.java | 45 ++ .../sql/jsonpath/tree/ContextVariable.java | 24 + .../sql/jsonpath/tree/DatetimeMethod.java | 41 ++ .../jsonpath/tree/DisjunctionPredicate.java | 45 ++ .../trino/sql/jsonpath/tree/DoubleMethod.java | 29 + .../sql/jsonpath/tree/ExistsPredicate.java | 38 ++ .../io/trino/sql/jsonpath/tree/Filter.java | 39 ++ .../trino/sql/jsonpath/tree/FloorMethod.java | 29 + .../sql/jsonpath/tree/IsUnknownPredicate.java | 38 ++ .../sql/jsonpath/tree/JsonNullLiteral.java | 28 + .../io/trino/sql/jsonpath/tree/JsonPath.java | 45 ++ .../jsonpath/tree/JsonPathTreeVisitor.java | 194 +++++++ .../sql/jsonpath/tree/KeyValueMethod.java | 29 + .../sql/jsonpath/tree/LastIndexVariable.java | 24 + .../sql/jsonpath/tree/LikeRegexPredicate.java | 54 ++ .../io/trino/sql/jsonpath/tree/Literal.java | 24 + .../sql/jsonpath/tree/MemberAccessor.java | 41 ++ .../io/trino/sql/jsonpath/tree/Method.java | 29 + .../sql/jsonpath/tree/NamedVariable.java | 38 ++ .../sql/jsonpath/tree/NegationPredicate.java | 38 ++ .../io/trino/sql/jsonpath/tree/PathNode.java | 22 + .../io/trino/sql/jsonpath/tree/Predicate.java | 24 + .../tree/PredicateCurrentItemVariable.java | 24 + .../trino/sql/jsonpath/tree/SizeMethod.java | 29 + .../sql/jsonpath/tree/SqlValueLiteral.java | 39 ++ .../jsonpath/tree/StartsWithPredicate.java | 52 ++ .../trino/sql/jsonpath/tree/TypeMethod.java | 29 + .../io/trino/sql/parser/ParsingException.java | 5 + .../sql/tree/DefaultTraversalVisitor.java | 37 ++ .../io/trino/sql/tree/ExpressionRewriter.java | 15 + .../sql/tree/ExpressionTreeRewriter.java | 98 ++++ .../java/io/trino/spi/StandardErrorCode.java | 4 + .../java/io/trino/spi/type/StandardTypes.java | 1 + 64 files changed, 4911 insertions(+), 3 deletions(-) create mode 100644 core/trino-main/src/main/java/io/trino/json/JsonInputErrorNode.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonExistsFunction.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonInputConversionError.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonInputFunctions.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonOutputConversionError.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonOutputFunctions.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonQueryFunction.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonValueFunction.java create mode 100644 core/trino-main/src/main/java/io/trino/sql/analyzer/JsonPathAnalyzer.java create mode 100644 core/trino-main/src/main/java/io/trino/type/Json2016Type.java create mode 100644 core/trino-main/src/main/java/io/trino/type/JsonPath2016Type.java create mode 100644 core/trino-parser/src/main/antlr4/io/trino/jsonpath/JsonPath.g4 create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathNodeRef.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathParser.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathTreeBuilder.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/AbsMethod.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Accessor.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArithmeticBinary.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArithmeticUnary.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArrayAccessor.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/CeilingMethod.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ComparisonPredicate.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ConjunctionPredicate.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ContextVariable.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DatetimeMethod.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DisjunctionPredicate.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DoubleMethod.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ExistsPredicate.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Filter.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/FloorMethod.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/IsUnknownPredicate.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonNullLiteral.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonPath.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonPathTreeVisitor.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/KeyValueMethod.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/LastIndexVariable.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/LikeRegexPredicate.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Literal.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/MemberAccessor.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Method.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/NamedVariable.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/NegationPredicate.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/PathNode.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Predicate.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/PredicateCurrentItemVariable.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/SizeMethod.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/SqlValueLiteral.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/StartsWithPredicate.java create mode 100644 core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/TypeMethod.java diff --git a/core/trino-main/src/main/java/io/trino/json/JsonInputErrorNode.java b/core/trino-main/src/main/java/io/trino/json/JsonInputErrorNode.java new file mode 100644 index 000000000000..bb6898c6c1c6 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/json/JsonInputErrorNode.java @@ -0,0 +1,169 @@ +/* + * 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 io.trino.json; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonPointer; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.ObjectCodec; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.jsontype.TypeSerializer; +import com.fasterxml.jackson.databind.node.JsonNodeType; + +import java.io.IOException; +import java.util.List; + +public class JsonInputErrorNode + extends JsonNode +{ + public static final JsonInputErrorNode JSON_ERROR = new JsonInputErrorNode(); + + private JsonInputErrorNode() {} + + @Override + public T deepCopy() + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonToken asToken() + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonParser.NumberType numberType() + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonNode get(int index) + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonNode path(String fieldName) + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonNode path(int index) + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonParser traverse() + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonParser traverse(ObjectCodec codec) + { + throw new UnsupportedOperationException(); + } + + @Override + protected JsonNode _at(JsonPointer ptr) + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonNodeType getNodeType() + { + throw new UnsupportedOperationException(); + } + + @Override + public String asText() + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonNode findValue(String fieldName) + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonNode findPath(String fieldName) + { + throw new UnsupportedOperationException(); + } + + @Override + public JsonNode findParent(String fieldName) + { + throw new UnsupportedOperationException(); + } + + @Override + public List findValues(String fieldName, List foundSoFar) + { + throw new UnsupportedOperationException(); + } + + @Override + public List findValuesAsText(String fieldName, List foundSoFar) + { + throw new UnsupportedOperationException(); + } + + @Override + public List findParents(String fieldName, List foundSoFar) + { + throw new UnsupportedOperationException(); + } + + @Override + public String toString() + { + return "JSON_ERROR"; + } + + @Override + public boolean equals(Object o) + { + return o == this; + } + + @Override + public int hashCode() + { + return getClass().hashCode(); + } + + @Override + public void serialize(JsonGenerator gen, SerializerProvider serializers) + throws IOException + { + throw new UnsupportedOperationException(); + } + + @Override + public void serializeWithType(JsonGenerator gen, SerializerProvider serializers, TypeSerializer typeSer) + throws IOException + { + throw new UnsupportedOperationException(); + } +} diff --git a/core/trino-main/src/main/java/io/trino/metadata/SystemFunctionBundle.java b/core/trino-main/src/main/java/io/trino/metadata/SystemFunctionBundle.java index d3e443bfafd0..ee08261726de 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/SystemFunctionBundle.java +++ b/core/trino-main/src/main/java/io/trino/metadata/SystemFunctionBundle.java @@ -157,6 +157,8 @@ import io.trino.operator.scalar.VersionFunction; import io.trino.operator.scalar.WilsonInterval; import io.trino.operator.scalar.WordStemFunction; +import io.trino.operator.scalar.json.JsonInputFunctions; +import io.trino.operator.scalar.json.JsonOutputFunctions; import io.trino.operator.scalar.time.LocalTimeFunction; import io.trino.operator.scalar.time.TimeFunctions; import io.trino.operator.scalar.time.TimeOperators; @@ -435,6 +437,8 @@ public static FunctionBundle create(FeaturesConfig featuresConfig, TypeOperators .scalars(DateTimeFunctions.class) .scalar(DateTimeFunctions.FromUnixtimeNanosDecimal.class) .scalars(JsonFunctions.class) + .scalars(JsonInputFunctions.class) + .scalars(JsonOutputFunctions.class) .scalars(ColorFunctions.class) .scalars(HyperLogLogFunctions.class) .scalars(QuantileDigestFunctions.class) diff --git a/core/trino-main/src/main/java/io/trino/metadata/TypeRegistry.java b/core/trino-main/src/main/java/io/trino/metadata/TypeRegistry.java index 8831054888f9..f2325a01b7b9 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/TypeRegistry.java +++ b/core/trino-main/src/main/java/io/trino/metadata/TypeRegistry.java @@ -94,6 +94,7 @@ import static io.trino.type.IntervalYearMonthType.INTERVAL_YEAR_MONTH; import static io.trino.type.IpAddressType.IPADDRESS; import static io.trino.type.JoniRegexpType.JONI_REGEXP; +import static io.trino.type.Json2016Type.JSON_2016; import static io.trino.type.JsonPathType.JSON_PATH; import static io.trino.type.JsonType.JSON; import static io.trino.type.LikePatternType.LIKE_PATTERN; @@ -146,6 +147,7 @@ public TypeRegistry(TypeOperators typeOperators, FeaturesConfig featuresConfig) addType(new Re2JRegexpType(featuresConfig.getRe2JDfaStatesLimit(), featuresConfig.getRe2JDfaRetries())); addType(LIKE_PATTERN); addType(JSON_PATH); + addType(JSON_2016); addType(COLOR); addType(JSON); addType(CODE_POINTS); diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonExistsFunction.java b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonExistsFunction.java new file mode 100644 index 000000000000..22fa55e7d537 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonExistsFunction.java @@ -0,0 +1,103 @@ +/* + * 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 io.trino.operator.scalar.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.trino.annotation.UsedByGeneratedCode; +import io.trino.metadata.BoundSignature; +import io.trino.metadata.FunctionManager; +import io.trino.metadata.FunctionMetadata; +import io.trino.metadata.Metadata; +import io.trino.metadata.Signature; +import io.trino.metadata.SqlScalarFunction; +import io.trino.operator.scalar.ChoicesScalarFunctionImplementation; +import io.trino.operator.scalar.JsonPath; +import io.trino.operator.scalar.ScalarFunctionImplementation; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; +import io.trino.spi.type.TypeSignature; +import io.trino.type.JsonPath2016Type; + +import java.lang.invoke.MethodHandle; + +import static io.trino.spi.function.InvocationConvention.InvocationArgumentConvention.BOXED_NULLABLE; +import static io.trino.spi.function.InvocationConvention.InvocationArgumentConvention.NEVER_NULL; +import static io.trino.spi.function.InvocationConvention.InvocationReturnConvention.NULLABLE_RETURN; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.StandardTypes.JSON_2016; +import static io.trino.spi.type.StandardTypes.TINYINT; +import static io.trino.util.Reflection.methodHandle; +import static java.util.Objects.requireNonNull; + +public class JsonExistsFunction + extends SqlScalarFunction +{ + public static final String JSON_EXISTS_FUNCTION_NAME = "$json_exists"; + private static final MethodHandle METHOD_HANDLE = methodHandle(JsonExistsFunction.class, "jsonExists", Metadata.class, TypeManager.class, Type.class, ConnectorSession.class, JsonNode.class, JsonPath.class, Object.class, long.class); // TODO replace JsonPath with IrJsonPath + + private final FunctionManager functionManager; + private final Metadata metadata; + private final TypeManager typeManager; + + public JsonExistsFunction(FunctionManager functionManager, Metadata metadata, TypeManager typeManager) + { + super(FunctionMetadata.scalarBuilder() + .signature(Signature.builder() + .name(JSON_EXISTS_FUNCTION_NAME) + .typeVariable("T") + .returnType(BOOLEAN) + .argumentTypes(ImmutableList.of(new TypeSignature(JSON_2016), new TypeSignature(JsonPath2016Type.NAME), new TypeSignature("T"), new TypeSignature(TINYINT))) + .build()) + .nullable() + .argumentNullability(false, false, true, false) + .hidden() + .description("Determines whether a JSON value satisfies a path specification") + .build()); + + this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + protected ScalarFunctionImplementation specialize(BoundSignature boundSignature) + { + Type parametersRowType = boundSignature.getArgumentType(2); + MethodHandle methodHandle = METHOD_HANDLE + .bindTo(metadata) + .bindTo(typeManager) + .bindTo(parametersRowType); + return new ChoicesScalarFunctionImplementation( + boundSignature, + NULLABLE_RETURN, + ImmutableList.of(BOXED_NULLABLE, BOXED_NULLABLE, BOXED_NULLABLE, NEVER_NULL), + methodHandle); + } + + @UsedByGeneratedCode + public static Boolean jsonExists( + Metadata metadata, + TypeManager typeManager, + Type parametersRowType, + ConnectorSession session, + JsonNode inputExpression, + JsonPath jsonPath, + Object parametersRow, + long errorBehavior) + { + throw new UnsupportedOperationException("JSON_EXISTS function is not yet supported"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonInputConversionError.java b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonInputConversionError.java new file mode 100644 index 000000000000..c230d4eb58da --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonInputConversionError.java @@ -0,0 +1,32 @@ +/* + * 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 io.trino.operator.scalar.json; + +import io.trino.spi.TrinoException; + +import static io.trino.spi.StandardErrorCode.JSON_INPUT_CONVERSION_ERROR; + +public class JsonInputConversionError + extends TrinoException +{ + public JsonInputConversionError(String message) + { + super(JSON_INPUT_CONVERSION_ERROR, "conversion to JSON failed: " + message); + } + + public JsonInputConversionError(Throwable cause) + { + super(JSON_INPUT_CONVERSION_ERROR, "conversion to JSON failed: ", cause); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonInputFunctions.java b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonInputFunctions.java new file mode 100644 index 000000000000..65419086ec6e --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonInputFunctions.java @@ -0,0 +1,114 @@ +/* + * 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 io.trino.operator.scalar.json; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.airlift.slice.Slice; +import io.trino.spi.TrinoException; +import io.trino.spi.function.ScalarFunction; +import io.trino.spi.function.SqlType; +import io.trino.spi.type.StandardTypes; + +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.Reader; +import java.nio.charset.Charset; + +import static io.trino.json.JsonInputErrorNode.JSON_ERROR; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static java.nio.charset.StandardCharsets.UTF_16LE; +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Read string input as JSON. + *

+ * These functions are used by JSON_EXISTS, JSON_VALUE and JSON_QUERY functions + * for parsing the JSON input arguments and applicable JSON path parameters. + *

+ * If the error handling strategy of the enclosing JSON function is ERROR ON ERROR, + * these input functions throw exception in case of parse error. + * Otherwise, the parse error is suppressed, and a marker value JSON_ERROR + * is returned, so that the enclosing function can handle the error accordingly + * to its error handling strategy (e.g. return a default value). + */ +public final class JsonInputFunctions +{ + public static final String VARCHAR_TO_JSON = "$varchar_to_json"; + public static final String VARBINARY_TO_JSON = "$varbinary_to_json"; + public static final String VARBINARY_UTF8_TO_JSON = "$varbinary_utf8_to_json"; + public static final String VARBINARY_UTF16_TO_JSON = "$varbinary_utf16_to_json"; + public static final String VARBINARY_UTF32_TO_JSON = "$varbinary_utf32_to_json"; + + private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final Charset UTF_32LE = Charset.forName("UTF-32LE"); + + private JsonInputFunctions() {} + + @ScalarFunction(value = VARCHAR_TO_JSON, hidden = true) + @SqlType(StandardTypes.JSON_2016) + public static JsonNode varcharToJson(@SqlType(StandardTypes.VARCHAR) Slice inputExpression, @SqlType(StandardTypes.BOOLEAN) boolean failOnError) + { + Reader reader = new InputStreamReader(inputExpression.getInput(), UTF_8); + return toJson(reader, failOnError); + } + + @ScalarFunction(value = VARBINARY_TO_JSON, hidden = true) + @SqlType(StandardTypes.JSON_2016) + public static JsonNode varbinaryToJson(@SqlType(StandardTypes.VARBINARY) Slice inputExpression, @SqlType(StandardTypes.BOOLEAN) boolean failOnError) + { + return varbinaryUtf8ToJson(inputExpression, failOnError); + } + + @ScalarFunction(value = VARBINARY_UTF8_TO_JSON, hidden = true) + @SqlType(StandardTypes.JSON_2016) + public static JsonNode varbinaryUtf8ToJson(@SqlType(StandardTypes.VARBINARY) Slice inputExpression, @SqlType(StandardTypes.BOOLEAN) boolean failOnError) + { + Reader reader = new InputStreamReader(inputExpression.getInput(), UTF_8); + return toJson(reader, failOnError); + } + + @ScalarFunction(value = VARBINARY_UTF16_TO_JSON, hidden = true) + @SqlType(StandardTypes.JSON_2016) + public static JsonNode varbinaryUtf16ToJson(@SqlType(StandardTypes.VARBINARY) Slice inputExpression, @SqlType(StandardTypes.BOOLEAN) boolean failOnError) + { + Reader reader = new InputStreamReader(inputExpression.getInput(), UTF_16LE); + return toJson(reader, failOnError); + } + + @ScalarFunction(value = VARBINARY_UTF32_TO_JSON, hidden = true) + @SqlType(StandardTypes.JSON_2016) + public static JsonNode varbinaryUtf32ToJson(@SqlType(StandardTypes.VARBINARY) Slice inputExpression, @SqlType(StandardTypes.BOOLEAN) boolean failOnError) + { + Reader reader = new InputStreamReader(inputExpression.getInput(), UTF_32LE); + return toJson(reader, failOnError); + } + + private static JsonNode toJson(Reader reader, boolean failOnError) + { + try { + return MAPPER.readTree(reader); + } + catch (JsonProcessingException e) { + if (failOnError) { + throw new JsonInputConversionError(e); + } + return JSON_ERROR; + } + catch (IOException e) { + throw new TrinoException(GENERIC_INTERNAL_ERROR, e); + } + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonOutputConversionError.java b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonOutputConversionError.java new file mode 100644 index 000000000000..3cec7b802337 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonOutputConversionError.java @@ -0,0 +1,32 @@ +/* + * 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 io.trino.operator.scalar.json; + +import io.trino.spi.TrinoException; + +import static io.trino.spi.StandardErrorCode.JSON_OUTPUT_CONVERSION_ERROR; + +public class JsonOutputConversionError + extends TrinoException +{ + public JsonOutputConversionError(String message) + { + super(JSON_OUTPUT_CONVERSION_ERROR, "conversion from JSON failed: " + message); + } + + public JsonOutputConversionError(Throwable cause) + { + super(JSON_OUTPUT_CONVERSION_ERROR, "conversion from JSON failed: ", cause); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonOutputFunctions.java b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonOutputFunctions.java new file mode 100644 index 000000000000..3ba498f8abdd --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonOutputFunctions.java @@ -0,0 +1,168 @@ +/* + * 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 io.trino.operator.scalar.json; + +import com.fasterxml.jackson.core.JsonEncoding; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.util.ByteArrayBuilder; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.trino.spi.TrinoException; +import io.trino.spi.function.ScalarFunction; +import io.trino.spi.function.SqlNullable; +import io.trino.spi.function.SqlType; +import io.trino.spi.type.StandardTypes; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; + +import static io.airlift.slice.Slices.wrappedBuffer; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.trino.sql.tree.JsonQuery.EmptyOrErrorBehavior.EMPTY_ARRAY; +import static io.trino.sql.tree.JsonQuery.EmptyOrErrorBehavior.EMPTY_OBJECT; +import static io.trino.sql.tree.JsonQuery.EmptyOrErrorBehavior.ERROR; +import static io.trino.sql.tree.JsonQuery.EmptyOrErrorBehavior.NULL; +import static java.util.Objects.requireNonNull; + +/** + * Format JSON as binary or character string, using given encoding. + *

+ * These functions are used to format the output of JSON_QUERY function. + * In case of error during JSON formatting, the error handling + * strategy of the enclosing JSON_QUERY function is applied. + *

+ * Additionally, the options KEEP / OMIT QUOTES [ON SCALAR STRING] + * are respected when formatting the output. + */ +public final class JsonOutputFunctions +{ + public static final String JSON_TO_VARCHAR = "$json_to_varchar"; + public static final String JSON_TO_VARBINARY = "$json_to_varbinary"; + public static final String JSON_TO_VARBINARY_UTF8 = "$json_to_varbinary_utf8"; + public static final String JSON_TO_VARBINARY_UTF16 = "$json_to_varbinary_utf16"; + public static final String JSON_TO_VARBINARY_UTF32 = "$json_to_varbinary_utf32"; + + private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final EncodingSpecificConstants UTF_8 = new EncodingSpecificConstants( + JsonEncoding.UTF8, + StandardCharsets.UTF_8, + Slices.copiedBuffer(new ArrayNode(JsonNodeFactory.instance).asText(), StandardCharsets.UTF_8), + Slices.copiedBuffer(new ObjectNode(JsonNodeFactory.instance).asText(), StandardCharsets.UTF_8)); + private static final EncodingSpecificConstants UTF_16 = new EncodingSpecificConstants( + JsonEncoding.UTF16_LE, + StandardCharsets.UTF_16LE, + Slices.copiedBuffer(new ArrayNode(JsonNodeFactory.instance).asText(), StandardCharsets.UTF_16LE), + Slices.copiedBuffer(new ObjectNode(JsonNodeFactory.instance).asText(), StandardCharsets.UTF_16LE)); + private static final EncodingSpecificConstants UTF_32 = new EncodingSpecificConstants( + JsonEncoding.UTF32_LE, + Charset.forName("UTF-32LE"), + Slices.copiedBuffer(new ArrayNode(JsonNodeFactory.instance).asText(), Charset.forName("UTF-32LE")), + Slices.copiedBuffer(new ObjectNode(JsonNodeFactory.instance).asText(), Charset.forName("UTF-32LE"))); + + private JsonOutputFunctions() {} + + @SqlNullable + @ScalarFunction(value = JSON_TO_VARCHAR, hidden = true) + @SqlType(StandardTypes.VARCHAR) + public static Slice jsonToVarchar(@SqlType(StandardTypes.JSON_2016) JsonNode jsonExpression, @SqlType(StandardTypes.TINYINT) long errorBehavior, @SqlType(StandardTypes.BOOLEAN) boolean omitQuotes) + { + return serialize(jsonExpression, UTF_8, errorBehavior, omitQuotes); + } + + @SqlNullable + @ScalarFunction(value = JSON_TO_VARBINARY, hidden = true) + @SqlType(StandardTypes.VARBINARY) + public static Slice jsonToVarbinary(@SqlType(StandardTypes.JSON_2016) JsonNode jsonExpression, @SqlType(StandardTypes.TINYINT) long errorBehavior, @SqlType(StandardTypes.BOOLEAN) boolean omitQuotes) + { + return jsonToVarbinaryUtf8(jsonExpression, errorBehavior, omitQuotes); + } + + @SqlNullable + @ScalarFunction(value = JSON_TO_VARBINARY_UTF8, hidden = true) + @SqlType(StandardTypes.VARBINARY) + public static Slice jsonToVarbinaryUtf8(@SqlType(StandardTypes.JSON_2016) JsonNode jsonExpression, @SqlType(StandardTypes.TINYINT) long errorBehavior, @SqlType(StandardTypes.BOOLEAN) boolean omitQuotes) + { + return serialize(jsonExpression, UTF_8, errorBehavior, omitQuotes); + } + + @SqlNullable + @ScalarFunction(value = JSON_TO_VARBINARY_UTF16, hidden = true) + @SqlType(StandardTypes.VARBINARY) + public static Slice jsonToVarbinaryUtf16(@SqlType(StandardTypes.JSON_2016) JsonNode jsonExpression, @SqlType(StandardTypes.TINYINT) long errorBehavior, @SqlType(StandardTypes.BOOLEAN) boolean omitQuotes) + { + return serialize(jsonExpression, UTF_16, errorBehavior, omitQuotes); + } + + @SqlNullable + @ScalarFunction(value = JSON_TO_VARBINARY_UTF32, hidden = true) + @SqlType(StandardTypes.VARBINARY) + public static Slice jsonToVarbinaryUtf32(@SqlType(StandardTypes.JSON_2016) JsonNode jsonExpression, @SqlType(StandardTypes.TINYINT) long errorBehavior, @SqlType(StandardTypes.BOOLEAN) boolean omitQuotes) + { + return serialize(jsonExpression, UTF_32, errorBehavior, omitQuotes); + } + + private static Slice serialize(JsonNode json, EncodingSpecificConstants constants, long errorBehavior, boolean omitQuotes) + { + if (omitQuotes && json.isTextual()) { + return Slices.copiedBuffer(json.asText(), constants.charset); + } + + ByteArrayBuilder builder = new ByteArrayBuilder(); + try (JsonGenerator generator = MAPPER.createGenerator(builder, constants.jsonEncoding)) { + MAPPER.writeTree(generator, json); + } + catch (JsonProcessingException e) { + if (errorBehavior == NULL.ordinal()) { + return null; + } + if (errorBehavior == ERROR.ordinal()) { + throw new JsonOutputConversionError(e); + } + if (errorBehavior == EMPTY_ARRAY.ordinal()) { + return constants.emptyArray; + } + if (errorBehavior == EMPTY_OBJECT.ordinal()) { + return constants.emptyObject; + } + throw new IllegalStateException("unexpected behavior"); + } + catch (IOException e) { + throw new TrinoException(GENERIC_INTERNAL_ERROR, e); + } + return wrappedBuffer(builder.toByteArray()); + } + + private static class EncodingSpecificConstants + { + private final JsonEncoding jsonEncoding; + private final Charset charset; + private final Slice emptyArray; + private final Slice emptyObject; + + public EncodingSpecificConstants(JsonEncoding jsonEncoding, Charset charset, Slice emptyArray, Slice emptyObject) + { + this.jsonEncoding = requireNonNull(jsonEncoding, "jsonEncoding is null"); + this.charset = requireNonNull(charset, "charset is null"); + this.emptyArray = requireNonNull(emptyArray, "emptyArray is null"); + this.emptyObject = requireNonNull(emptyObject, "emptyObject is null"); + } + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonQueryFunction.java b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonQueryFunction.java new file mode 100644 index 000000000000..a5271312b8e8 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonQueryFunction.java @@ -0,0 +1,110 @@ +/* + * 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 io.trino.operator.scalar.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.trino.annotation.UsedByGeneratedCode; +import io.trino.metadata.BoundSignature; +import io.trino.metadata.FunctionManager; +import io.trino.metadata.FunctionMetadata; +import io.trino.metadata.Metadata; +import io.trino.metadata.Signature; +import io.trino.metadata.SqlScalarFunction; +import io.trino.operator.scalar.ChoicesScalarFunctionImplementation; +import io.trino.operator.scalar.JsonPath; +import io.trino.operator.scalar.ScalarFunctionImplementation; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; +import io.trino.spi.type.TypeSignature; +import io.trino.type.JsonPath2016Type; + +import java.lang.invoke.MethodHandle; + +import static io.trino.spi.function.InvocationConvention.InvocationArgumentConvention.BOXED_NULLABLE; +import static io.trino.spi.function.InvocationConvention.InvocationArgumentConvention.NEVER_NULL; +import static io.trino.spi.function.InvocationConvention.InvocationReturnConvention.NULLABLE_RETURN; +import static io.trino.spi.type.StandardTypes.JSON_2016; +import static io.trino.spi.type.StandardTypes.TINYINT; +import static io.trino.util.Reflection.methodHandle; +import static java.util.Objects.requireNonNull; + +public class JsonQueryFunction + extends SqlScalarFunction +{ + public static final String JSON_QUERY_FUNCTION_NAME = "$json_query"; + private static final MethodHandle METHOD_HANDLE = methodHandle(JsonQueryFunction.class, "jsonQuery", Metadata.class, TypeManager.class, Type.class, ConnectorSession.class, JsonNode.class, JsonPath.class, Object.class, long.class, long.class, long.class); // TODO replace JsonPath with IrJsonPath + + private final FunctionManager functionManager; + private final Metadata metadata; + private final TypeManager typeManager; + + public JsonQueryFunction(FunctionManager functionManager, Metadata metadata, TypeManager typeManager) + { + super(FunctionMetadata.scalarBuilder() + .signature(Signature.builder() + .name(JSON_QUERY_FUNCTION_NAME) + .typeVariable("T") + .returnType(new TypeSignature(JSON_2016)) + .argumentTypes(ImmutableList.of( + new TypeSignature(JSON_2016), + new TypeSignature(JsonPath2016Type.NAME), + new TypeSignature("T"), + new TypeSignature(TINYINT), + new TypeSignature(TINYINT), + new TypeSignature(TINYINT))) + .build()) + .nullable() + .argumentNullability(false, false, true, false, false, false) + .hidden() + .description("Extracts a JSON value from a JSON value") + .build()); + + this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + protected ScalarFunctionImplementation specialize(BoundSignature boundSignature) + { + Type parametersRowType = boundSignature.getArgumentType(2); + MethodHandle methodHandle = METHOD_HANDLE + .bindTo(metadata) + .bindTo(typeManager) + .bindTo(parametersRowType); + return new ChoicesScalarFunctionImplementation( + boundSignature, + NULLABLE_RETURN, + ImmutableList.of(BOXED_NULLABLE, BOXED_NULLABLE, BOXED_NULLABLE, NEVER_NULL, NEVER_NULL, NEVER_NULL), + methodHandle); + } + + @UsedByGeneratedCode + public static JsonNode jsonQuery( + Metadata metadata, + TypeManager typeManager, + Type parametersRowType, + ConnectorSession session, + JsonNode inputExpression, + JsonPath jsonPath, + Object parametersRow, + long wrapperBehavior, + long emptyBehavior, + long errorBehavior) + { + throw new UnsupportedOperationException("JSON_QUERY function is not yet supported"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonValueFunction.java b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonValueFunction.java new file mode 100644 index 000000000000..7cefb1c6f0d9 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/json/JsonValueFunction.java @@ -0,0 +1,116 @@ +/* + * 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 io.trino.operator.scalar.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.trino.annotation.UsedByGeneratedCode; +import io.trino.metadata.BoundSignature; +import io.trino.metadata.FunctionManager; +import io.trino.metadata.FunctionMetadata; +import io.trino.metadata.Metadata; +import io.trino.metadata.Signature; +import io.trino.metadata.SqlScalarFunction; +import io.trino.operator.scalar.ChoicesScalarFunctionImplementation; +import io.trino.operator.scalar.JsonPath; +import io.trino.operator.scalar.ScalarFunctionImplementation; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; +import io.trino.spi.type.TypeSignature; +import io.trino.type.JsonPath2016Type; + +import java.lang.invoke.MethodHandle; + +import static io.trino.spi.function.InvocationConvention.InvocationArgumentConvention.BOXED_NULLABLE; +import static io.trino.spi.function.InvocationConvention.InvocationArgumentConvention.NEVER_NULL; +import static io.trino.spi.function.InvocationConvention.InvocationReturnConvention.NULLABLE_RETURN; +import static io.trino.spi.type.StandardTypes.JSON_2016; +import static io.trino.spi.type.StandardTypes.TINYINT; +import static io.trino.util.Reflection.methodHandle; +import static java.util.Objects.requireNonNull; + +public class JsonValueFunction + extends SqlScalarFunction +{ + public static final String JSON_VALUE_FUNCTION_NAME = "$json_value"; + private static final MethodHandle METHOD_HANDLE = methodHandle(JsonValueFunction.class, "jsonValue", Metadata.class, TypeManager.class, Type.class, Type.class, ConnectorSession.class, JsonNode.class, JsonPath.class, Object.class, long.class, Object.class, long.class, Object.class); // TODO replace JsonPath with IrJsonPath + + private final FunctionManager functionManager; + private final Metadata metadata; + private final TypeManager typeManager; + + public JsonValueFunction(FunctionManager functionManager, Metadata metadata, TypeManager typeManager) + { + super(FunctionMetadata.scalarBuilder() + .signature(Signature.builder() + .name(JSON_VALUE_FUNCTION_NAME) + .typeVariable("R") + .typeVariable("T") + .returnType(new TypeSignature("R")) + .argumentTypes(ImmutableList.of( + new TypeSignature(JSON_2016), + new TypeSignature(JsonPath2016Type.NAME), + new TypeSignature("T"), + new TypeSignature(TINYINT), + new TypeSignature("R"), + new TypeSignature(TINYINT), + new TypeSignature("R"))) + .build()) + .nullable() + .argumentNullability(false, false, true, false, true, false, true) + .hidden() + .description("Extracts an SQL scalar from a JSON value") + .build()); + + this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + protected ScalarFunctionImplementation specialize(BoundSignature boundSignature) + { + Type parametersRowType = boundSignature.getArgumentType(2); + Type returnType = boundSignature.getArgumentType(4); + MethodHandle methodHandle = METHOD_HANDLE + .bindTo(metadata) + .bindTo(typeManager) + .bindTo(parametersRowType) + .bindTo(returnType); + return new ChoicesScalarFunctionImplementation( + boundSignature, + NULLABLE_RETURN, + ImmutableList.of(BOXED_NULLABLE, BOXED_NULLABLE, BOXED_NULLABLE, NEVER_NULL, BOXED_NULLABLE, NEVER_NULL, BOXED_NULLABLE), + methodHandle); + } + + @UsedByGeneratedCode + public static Object jsonValue( + Metadata metadata, + TypeManager typeManager, + Type parametersRowType, + Type returnType, + ConnectorSession session, + JsonNode inputExpression, + JsonPath jsonPath, + Object parametersRow, + long emptyBehavior, + Object emptyDefault, + long errorBehavior, + Object errorDefault) + { + throw new UnsupportedOperationException("JSON_VALUE function is not yet supported"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java index 060d2117e1d8..2e4fc888880b 100644 --- a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java +++ b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java @@ -101,6 +101,9 @@ import io.trino.operator.PagesIndexPageSorter; import io.trino.operator.TrinoOperatorFactories; import io.trino.operator.index.IndexJoinLookupStats; +import io.trino.operator.scalar.json.JsonExistsFunction; +import io.trino.operator.scalar.json.JsonQueryFunction; +import io.trino.operator.scalar.json.JsonValueFunction; import io.trino.server.ExpressionSerialization.ExpressionDeserializer; import io.trino.server.ExpressionSerialization.ExpressionSerializer; import io.trino.server.PluginManager.PluginsProvider; @@ -150,6 +153,7 @@ import io.trino.transaction.TransactionManagerConfig; import io.trino.type.BlockTypeOperators; import io.trino.type.InternalTypeManager; +import io.trino.type.JsonPath2016Type; import io.trino.type.TypeDeserializer; import io.trino.type.TypeOperatorsCache; import io.trino.type.TypeSignatureDeserializer; @@ -408,6 +412,7 @@ protected void setup(Binder binder) binder.bind(TypeRegistry.class).in(Scopes.SINGLETON); binder.bind(TypeManager.class).to(InternalTypeManager.class).in(Scopes.SINGLETON); newSetBinder(binder, Type.class); + binder.bind(RegisterJsonPath2016Type.class).asEagerSingleton(); // split manager binder.bind(SplitManager.class).in(Scopes.SINGLETON); @@ -526,6 +531,27 @@ public static FunctionBundle literalFunctionBundle(BlockEncodingSerde blockEncod return new InternalFunctionBundle(new LiteralFunction(blockEncodingSerde)); } + @ProvidesIntoSet + @Singleton + // not adding to system function bundle to avoid mutual dependency FunctionManager <-> MetadataManager in testing instance constructors + public static FunctionBundle jsonFunctionBundle(FunctionManager functionManager, Metadata metadata, TypeManager typeManager) + { + return new InternalFunctionBundle( + new JsonExistsFunction(functionManager, metadata, typeManager), + new JsonValueFunction(functionManager, metadata, typeManager), + new JsonQueryFunction(functionManager, metadata, typeManager)); + } + + // working around circular dependency Type <-> TypeManager + private static class RegisterJsonPath2016Type + { + @Inject + public RegisterJsonPath2016Type(BlockEncodingSerde blockEncodingSerde, TypeManager typeManager, TypeRegistry typeRegistry) + { + typeRegistry.addType(new JsonPath2016Type(new TypeDeserializer(typeManager), blockEncodingSerde)); + } + } + @Provides @Singleton public static TypeOperators createTypeOperators(TypeOperatorsCache typeOperatorsCache) diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/AggregationAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/AggregationAnalyzer.java index bb462c042aa3..244e8719a2ac 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/AggregationAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/AggregationAnalyzer.java @@ -43,6 +43,11 @@ import io.trino.sql.tree.InPredicate; import io.trino.sql.tree.IsNotNullPredicate; import io.trino.sql.tree.IsNullPredicate; +import io.trino.sql.tree.JsonExists; +import io.trino.sql.tree.JsonPathInvocation; +import io.trino.sql.tree.JsonPathParameter; +import io.trino.sql.tree.JsonQuery; +import io.trino.sql.tree.JsonValue; import io.trino.sql.tree.LambdaExpression; import io.trino.sql.tree.LikePredicate; import io.trino.sql.tree.Literal; @@ -715,6 +720,35 @@ protected Boolean visitGroupingOperation(GroupingOperation node, Void context) return true; } + @Override + protected Boolean visitJsonExists(JsonExists node, Void context) + { + return process(node.getJsonPathInvocation(), context); + } + + @Override + protected Boolean visitJsonValue(JsonValue node, Void context) + { + return process(node.getJsonPathInvocation(), context) && + node.getEmptyDefault().map(expression -> process(expression, context)).orElse(true) && + node.getErrorDefault().map(expression -> process(expression, context)).orElse(true); + } + + @Override + protected Boolean visitJsonQuery(JsonQuery node, Void context) + { + return process(node.getJsonPathInvocation(), context); + } + + @Override + protected Boolean visitJsonPathInvocation(JsonPathInvocation node, Void context) + { + return process(node.getInputExpression(), context) && + node.getPathParameters().stream() + .map(JsonPathParameter::getParameter) + .allMatch(expression -> process(expression, context)); + } + @Override public Boolean process(Node node, @Nullable Void context) { diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java index 8c630b74ced1..7a7490ab6a2d 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java @@ -45,6 +45,7 @@ import io.trino.spi.security.Identity; import io.trino.spi.type.Type; import io.trino.sql.analyzer.ExpressionAnalyzer.LabelPrefixedReference; +import io.trino.sql.analyzer.JsonPathAnalyzer.JsonPathAnalysis; import io.trino.sql.tree.AllColumns; import io.trino.sql.tree.DereferenceExpression; import io.trino.sql.tree.ExistsPredicate; @@ -152,6 +153,11 @@ public class Analysis private final Set> patternAggregations = new LinkedHashSet<>(); + // for JSON features + private final Map, JsonPathAnalysis> jsonPathAnalyses = new LinkedHashMap<>(); + private final Map, ResolvedFunction> jsonInputFunctions = new LinkedHashMap<>(); + private final Map, ResolvedFunction> jsonOutputFunctions = new LinkedHashMap<>(); + private final Map, List> aggregates = new LinkedHashMap<>(); private final Map, List> orderByAggregates = new LinkedHashMap<>(); private final Map, GroupingSetAnalysis> groupingSets = new LinkedHashMap<>(); @@ -980,6 +986,36 @@ public boolean isPatternAggregation(FunctionCall function) return patternAggregations.contains(NodeRef.of(function)); } + public void setJsonPathAnalyses(Map, JsonPathAnalysis> pathAnalyses) + { + jsonPathAnalyses.putAll(pathAnalyses); + } + + public JsonPathAnalysis getJsonPathAnalysis(Expression expression) + { + return jsonPathAnalyses.get(NodeRef.of(expression)); + } + + public void setJsonInputFunctions(Map, ResolvedFunction> functions) + { + jsonInputFunctions.putAll(functions); + } + + public ResolvedFunction getJsonInputFunction(Expression expression) + { + return jsonInputFunctions.get(NodeRef.of(expression)); + } + + public void setJsonOutputFunctions(Map, ResolvedFunction> functions) + { + jsonOutputFunctions.putAll(functions); + } + + public ResolvedFunction getJsonOutputFunction(Expression expression) + { + return jsonOutputFunctions.get(NodeRef.of(expression)); + } + public Map>> getTableColumnReferences() { return tableColumnReferences; diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java index 667067f868f5..4ab4e8a05ed3 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java @@ -48,6 +48,7 @@ import io.trino.spi.type.TimestampType; import io.trino.spi.type.TimestampWithTimeZoneType; import io.trino.spi.type.Type; +import io.trino.spi.type.TypeId; import io.trino.spi.type.TypeNotFoundException; import io.trino.spi.type.TypeSignatureParameter; import io.trino.spi.type.VarcharType; @@ -55,6 +56,7 @@ import io.trino.sql.analyzer.Analysis.PredicateCoercions; import io.trino.sql.analyzer.Analysis.Range; import io.trino.sql.analyzer.Analysis.ResolvedWindow; +import io.trino.sql.analyzer.JsonPathAnalyzer.JsonPathAnalysis; import io.trino.sql.analyzer.PatternRecognitionAnalyzer.PatternRecognitionAnalysis; import io.trino.sql.planner.LiteralInterpreter; import io.trino.sql.planner.Symbol; @@ -95,6 +97,12 @@ import io.trino.sql.tree.IntervalLiteral; import io.trino.sql.tree.IsNotNullPredicate; import io.trino.sql.tree.IsNullPredicate; +import io.trino.sql.tree.JsonExists; +import io.trino.sql.tree.JsonPathInvocation; +import io.trino.sql.tree.JsonPathParameter; +import io.trino.sql.tree.JsonPathParameter.JsonFormat; +import io.trino.sql.tree.JsonQuery; +import io.trino.sql.tree.JsonValue; import io.trino.sql.tree.LambdaArgumentDeclaration; import io.trino.sql.tree.LambdaExpression; import io.trino.sql.tree.LikePredicate; @@ -132,6 +140,7 @@ import io.trino.sql.tree.WindowFrame; import io.trino.sql.tree.WindowOperation; import io.trino.type.FunctionType; +import io.trino.type.JsonPath2016Type; import io.trino.type.TypeCoercion; import io.trino.type.UnknownType; @@ -158,8 +167,22 @@ import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.collect.cache.CacheUtils.uncheckedCacheGet; import static io.trino.collect.cache.SafeCaches.buildNonEvictableCache; +import static io.trino.operator.scalar.json.JsonExistsFunction.JSON_EXISTS_FUNCTION_NAME; +import static io.trino.operator.scalar.json.JsonInputFunctions.VARBINARY_TO_JSON; +import static io.trino.operator.scalar.json.JsonInputFunctions.VARBINARY_UTF16_TO_JSON; +import static io.trino.operator.scalar.json.JsonInputFunctions.VARBINARY_UTF32_TO_JSON; +import static io.trino.operator.scalar.json.JsonInputFunctions.VARBINARY_UTF8_TO_JSON; +import static io.trino.operator.scalar.json.JsonInputFunctions.VARCHAR_TO_JSON; +import static io.trino.operator.scalar.json.JsonOutputFunctions.JSON_TO_VARBINARY; +import static io.trino.operator.scalar.json.JsonOutputFunctions.JSON_TO_VARBINARY_UTF16; +import static io.trino.operator.scalar.json.JsonOutputFunctions.JSON_TO_VARBINARY_UTF32; +import static io.trino.operator.scalar.json.JsonOutputFunctions.JSON_TO_VARBINARY_UTF8; +import static io.trino.operator.scalar.json.JsonOutputFunctions.JSON_TO_VARCHAR; +import static io.trino.operator.scalar.json.JsonQueryFunction.JSON_QUERY_FUNCTION_NAME; +import static io.trino.operator.scalar.json.JsonValueFunction.JSON_VALUE_FUNCTION_NAME; import static io.trino.spi.StandardErrorCode.AMBIGUOUS_NAME; import static io.trino.spi.StandardErrorCode.COLUMN_NOT_FOUND; +import static io.trino.spi.StandardErrorCode.DUPLICATE_PARAMETER_NAME; import static io.trino.spi.StandardErrorCode.EXPRESSION_NOT_CONSTANT; import static io.trino.spi.StandardErrorCode.FUNCTION_NOT_AGGREGATE; import static io.trino.spi.StandardErrorCode.INVALID_ARGUMENTS; @@ -221,6 +244,9 @@ import static io.trino.sql.tree.FrameBound.Type.PRECEDING; import static io.trino.sql.tree.FrameBound.Type.UNBOUNDED_FOLLOWING; import static io.trino.sql.tree.FrameBound.Type.UNBOUNDED_PRECEDING; +import static io.trino.sql.tree.JsonQuery.ArrayWrapperBehavior.CONDITIONAL; +import static io.trino.sql.tree.JsonQuery.ArrayWrapperBehavior.UNCONDITIONAL; +import static io.trino.sql.tree.JsonValue.EmptyOrErrorBehavior.DEFAULT; import static io.trino.sql.tree.SortItem.Ordering.ASCENDING; import static io.trino.sql.tree.SortItem.Ordering.DESCENDING; import static io.trino.sql.tree.WindowFrame.Type.GROUPS; @@ -237,6 +263,7 @@ import static io.trino.type.DateTimes.timestampHasTimeZone; import static io.trino.type.IntervalDayTimeType.INTERVAL_DAY_TIME; import static io.trino.type.IntervalYearMonthType.INTERVAL_YEAR_MONTH; +import static io.trino.type.Json2016Type.JSON_2016; import static io.trino.type.JsonType.JSON; import static io.trino.type.UnknownType.UNKNOWN; import static java.lang.Math.toIntExact; @@ -251,6 +278,8 @@ public class ExpressionAnalyzer private static final int MAX_NUMBER_GROUPING_ARGUMENTS_BIGINT = 63; private static final int MAX_NUMBER_GROUPING_ARGUMENTS_INTEGER = 31; + public static final RowType JSON_NO_PARAMETERS_ROW_TYPE = RowType.anonymous(ImmutableList.of(UNKNOWN)); + private final PlannerContext plannerContext; private final AccessControl accessControl; private final BiFunction statementAnalyzerFactory; @@ -299,6 +328,11 @@ public class ExpressionAnalyzer private final Map, MeasureDefinition> measureDefinitions = new LinkedHashMap<>(); private final Set> patternAggregations = new LinkedHashSet<>(); + // for JSON functions + private final Map, JsonPathAnalysis> jsonPathAnalyses = new LinkedHashMap<>(); + private final Map, ResolvedFunction> jsonInputFunctions = new LinkedHashMap<>(); + private final Map, ResolvedFunction> jsonOutputFunctions = new LinkedHashMap<>(); + private final Session session; private final Map, Expression> parameters; private final WarningCollector warningCollector; @@ -523,6 +557,21 @@ public Set> getPatternAggregations() return patternAggregations; } + public Map, JsonPathAnalysis> getJsonPathAnalyses() + { + return jsonPathAnalyses; + } + + public Map, ResolvedFunction> getJsonInputFunctions() + { + return jsonInputFunctions; + } + + public Map, ResolvedFunction> getJsonOutputFunctions() + { + return jsonOutputFunctions; + } + private class Visitor extends StackableAstVisitor { @@ -1767,8 +1816,8 @@ private ArgumentLabel validateLabelConsistency(FunctionCall node, boolean labelR String name = node.getName().getSuffix(); List unlabeledInputColumns = Streams.concat( - extractExpressions(ImmutableList.of(node.getArguments().get(argumentIndex)), Identifier.class).stream(), - extractExpressions(ImmutableList.of(node.getArguments().get(argumentIndex)), DereferenceExpression.class).stream()) + extractExpressions(ImmutableList.of(node.getArguments().get(argumentIndex)), Identifier.class).stream(), + extractExpressions(ImmutableList.of(node.getArguments().get(argumentIndex)), DereferenceExpression.class).stream()) .filter(expression -> columnReferences.containsKey(NodeRef.of(expression))) .collect(toImmutableList()); List labeledInputColumns = extractExpressions(ImmutableList.of(node.getArguments().get(argumentIndex)), DereferenceExpression.class).stream() @@ -2496,6 +2545,370 @@ public Type visitGroupingOperation(GroupingOperation node, StackableAstVisitorCo } } + @Override + public Type visitJsonExists(JsonExists node, StackableAstVisitorContext context) + { + List pathInvocationArgumentTypes = analyzeJsonPathInvocation("JSON_EXISTS", node, node.getJsonPathInvocation(), context); + + // pass remaining information in the node : error behavior + List argumentTypes = ImmutableList.builder() + .addAll(pathInvocationArgumentTypes) + .add(TINYINT) // enum encoded as integer value + .build(); + + // resolve function + ResolvedFunction function; + try { + function = plannerContext.getMetadata().resolveFunction(session, QualifiedName.of(JSON_EXISTS_FUNCTION_NAME), fromTypes(argumentTypes)); + } + catch (TrinoException e) { + if (e.getLocation().isPresent()) { + throw e; + } + throw new TrinoException(e::getErrorCode, extractLocation(node), e.getMessage(), e); + } + accessControl.checkCanExecuteFunction(SecurityContext.of(session), JSON_EXISTS_FUNCTION_NAME); + resolvedFunctions.put(NodeRef.of(node), function); + Type type = function.getSignature().getReturnType(); + + return setExpressionType(node, type); + } + + @Override + public Type visitJsonValue(JsonValue node, StackableAstVisitorContext context) + { + List pathInvocationArgumentTypes = analyzeJsonPathInvocation("JSON_VALUE", node, node.getJsonPathInvocation(), context); + + // validate returned type + Type returnedType = VARCHAR; // default + if (node.getReturnedType().isPresent()) { + try { + returnedType = plannerContext.getTypeManager().getType(toTypeSignature(node.getReturnedType().get())); + } + catch (TypeNotFoundException e) { + throw semanticException(TYPE_MISMATCH, node, "Unknown type: %s", node.getReturnedType().get()); + } + } + + if (!isCharacterStringType(returnedType) && + !isNumericType(returnedType) && + !returnedType.equals(BOOLEAN) && + !isDateTimeType(returnedType) || + returnedType.equals(INTERVAL_DAY_TIME) || + returnedType.equals(INTERVAL_YEAR_MONTH)) { + throw semanticException(TYPE_MISMATCH, node, "Invalid return type of function JSON_VALUE: " + node.getReturnedType().get()); + } + + JsonPathAnalysis pathAnalysis = jsonPathAnalyses.get(NodeRef.of(node)); + Type resultType = pathAnalysis.getType(pathAnalysis.getPath()); + if (resultType != null && !resultType.equals(returnedType)) { + try { + plannerContext.getMetadata().getCoercion(session, resultType, returnedType); + } + catch (OperatorNotFoundException e) { + throw semanticException(TYPE_MISMATCH, node, "Return type of JSON path: %s incompatible with return type of function JSON_VALUE: %s", resultType, returnedType); + } + } + + // validate default values for empty and error behavior + if (node.getEmptyDefault().isPresent()) { + Expression emptyDefault = node.getEmptyDefault().get(); + if (node.getEmptyBehavior() != DEFAULT) { + throw semanticException(INVALID_FUNCTION_ARGUMENT, emptyDefault, "Default value specified for %s ON EMPTY behavior", node.getEmptyBehavior()); + } + Type type = process(emptyDefault, context); + // this would normally be done after function resolution, but we know that the default expression is always coerced to the returnedType + coerceType(emptyDefault, type, returnedType, "Function JSON_VALUE default ON EMPTY result"); + } + + if (node.getErrorDefault().isPresent()) { + Expression errorDefault = node.getErrorDefault().get(); + if (node.getErrorBehavior() != DEFAULT) { + throw semanticException(INVALID_FUNCTION_ARGUMENT, errorDefault, "Default value specified for %s ON ERROR behavior", node.getErrorBehavior()); + } + Type type = process(errorDefault, context); + // this would normally be done after function resolution, but we know that the default expression is always coerced to the returnedType + coerceType(errorDefault, type, returnedType, "Function JSON_VALUE default ON ERROR result"); + } + + // pass remaining information in the node : empty behavior, empty default, error behavior, error default + List argumentTypes = ImmutableList.builder() + .addAll(pathInvocationArgumentTypes) + .add(TINYINT) // empty behavior: enum encoded as integer value + .add(returnedType) // empty default + .add(TINYINT) // error behavior: enum encoded as integer value + .add(returnedType) // error default + .build(); + + // resolve function + ResolvedFunction function; + try { + function = plannerContext.getMetadata().resolveFunction(session, QualifiedName.of(JSON_VALUE_FUNCTION_NAME), fromTypes(argumentTypes)); + } + catch (TrinoException e) { + if (e.getLocation().isPresent()) { + throw e; + } + throw new TrinoException(e::getErrorCode, extractLocation(node), e.getMessage(), e); + } + + accessControl.checkCanExecuteFunction(SecurityContext.of(session), JSON_VALUE_FUNCTION_NAME); + resolvedFunctions.put(NodeRef.of(node), function); + Type type = function.getSignature().getReturnType(); + + return setExpressionType(node, type); + } + + @Override + public Type visitJsonQuery(JsonQuery node, StackableAstVisitorContext context) + { + List pathInvocationArgumentTypes = analyzeJsonPathInvocation("JSON_QUERY", node, node.getJsonPathInvocation(), context); + + // validate wrapper and quotes behavior + if ((node.getWrapperBehavior() == CONDITIONAL || node.getWrapperBehavior() == UNCONDITIONAL) && node.getQuotesBehavior().isPresent()) { + throw semanticException(INVALID_FUNCTION_ARGUMENT, node, "%s QUOTES behavior specified with WITH %s ARRAY WRAPPER behavior", node.getQuotesBehavior().get(), node.getWrapperBehavior()); + } + + // wrapper behavior, empty behavior and error behavior will be passed as arguments to function + // quotes behavior is handled by the corresponding output function + List argumentTypes = ImmutableList.builder() + .addAll(pathInvocationArgumentTypes) + .add(TINYINT) // wrapper behavior: enum encoded as integer value + .add(TINYINT) // empty behavior: enum encoded as integer value + .add(TINYINT) // error behavior: enum encoded as integer value + .build(); + + // resolve function + ResolvedFunction function; + try { + function = plannerContext.getMetadata().resolveFunction(session, QualifiedName.of(JSON_QUERY_FUNCTION_NAME), fromTypes(argumentTypes)); + } + catch (TrinoException e) { + if (e.getLocation().isPresent()) { + throw e; + } + throw new TrinoException(e::getErrorCode, extractLocation(node), e.getMessage(), e); + } + accessControl.checkCanExecuteFunction(SecurityContext.of(session), JSON_QUERY_FUNCTION_NAME); + resolvedFunctions.put(NodeRef.of(node), function); + + // analyze returned type and format + Type returnedType = VARCHAR; // default + if (node.getReturnedType().isPresent()) { + try { + returnedType = plannerContext.getTypeManager().getType(toTypeSignature(node.getReturnedType().get())); + } + catch (TypeNotFoundException e) { + throw semanticException(TYPE_MISMATCH, node, "Unknown type: %s", node.getReturnedType().get()); + } + } + JsonFormat outputFormat = node.getOutputFormat().orElse(JsonFormat.JSON); // default + + // resolve function to format output + ResolvedFunction outputFunction = getOutputFunction(returnedType, outputFormat, node); + jsonOutputFunctions.put(NodeRef.of(node), outputFunction); + + // cast the output value to the declared returned type if necessary + Type outputType = outputFunction.getSignature().getReturnType(); + if (!outputType.equals(returnedType)) { + try { + plannerContext.getMetadata().getCoercion(session, outputType, returnedType); + } + catch (OperatorNotFoundException e) { + throw semanticException(TYPE_MISMATCH, node, "Cannot cast %s to %s", outputType, returnedType); + } + } + + return setExpressionType(node, returnedType); + } + + private List analyzeJsonPathInvocation(String functionName, Expression node, JsonPathInvocation jsonPathInvocation, StackableAstVisitorContext context) + { + // ANALYZE THE CONTEXT ITEM + // analyze context item type + Expression inputExpression = jsonPathInvocation.getInputExpression(); + Type inputType = process(inputExpression, context); + + // resolve function to read the context item as JSON + JsonFormat inputFormat = jsonPathInvocation.getInputFormat(); + ResolvedFunction inputFunction = getInputFunction(inputType, inputFormat, inputExpression); + Type expectedType = inputFunction.getSignature().getArgumentType(0); + coerceType(inputExpression, inputType, expectedType, format("%s function input argument", functionName)); + jsonInputFunctions.put(NodeRef.of(inputExpression), inputFunction); + + // ANALYZE JSON PATH PARAMETERS + // TODO verify parameter count? Is there a limit on Row size? + + ImmutableMap.Builder types = ImmutableMap.builder(); // record parameter types for JSON path analysis + Set uniqueNames = new HashSet<>(); // validate parameter names + + // this node will be translated into a FunctionCall, and all the information it carries will be passed as arguments to the FunctionCall. + // all JSON path parameters are wrapped in a Row, and constitute a single FunctionCall argument. + ImmutableList.Builder fields = ImmutableList.builder(); + + List pathParameters = jsonPathInvocation.getPathParameters(); + for (JsonPathParameter pathParameter : pathParameters) { + Expression parameter = pathParameter.getParameter(); + String parameterName = pathParameter.getName().getCanonicalValue(); + Optional parameterFormat = pathParameter.getFormat(); + + // type of the parameter passed to the JSON path: + // - parameters of types numeric, string, boolean, date,... are passed as-is + // - parameters with explicit or implicit FORMAT, are converted to JSON (type JSON_2016) + // - all other parameters are cast to VARCHAR + Type passedType; + + if (!uniqueNames.add(parameterName)) { + throw semanticException(DUPLICATE_PARAMETER_NAME, pathParameter.getName(), "%s JSON path parameter is specified more than once", parameterName); + } + + if (parameter instanceof LambdaExpression || parameter instanceof BindExpression) { + throw semanticException(NOT_SUPPORTED, parameter, "%s is not supported as JSON path parameter", parameter.getClass().getSimpleName()); + } + // if the input expression is a JSON-returning function, there should be an explicit or implicit input format (spec p.817) + // JSON-returning functions are: JSON_OBJECT, JSON_OBJECTAGG, JSON_ARRAY, JSON_ARRAYAGG and JSON_QUERY + if (parameter instanceof JsonQuery && // TODO add JSON_OBJECT, JSON_OBJECTAGG, JSON_ARRAY, JSON_ARRAYAGG when supported + parameterFormat.isEmpty()) { + parameterFormat = Optional.of(JsonFormat.JSON); + } + + Type parameterType = process(parameter, context); + if (parameterFormat.isPresent()) { + // resolve function to read the parameter as JSON + ResolvedFunction parameterInputFunction = getInputFunction(parameterType, parameterFormat.get(), parameter); + Type expectedParameterType = parameterInputFunction.getSignature().getArgumentType(0); + coerceType(parameter, parameterType, expectedParameterType, format("%s function JSON path parameter", functionName)); + jsonInputFunctions.put(NodeRef.of(parameter), parameterInputFunction); + passedType = JSON_2016; + } + else { + if (isStringType(parameterType)) { + if (!isCharacterStringType(parameterType)) { + throw semanticException(NOT_SUPPORTED, parameter, "Unsupported type of JSON path parameter: %s", parameterType.getDisplayName()); + } + passedType = parameterType; + } + else if (isNumericType(parameterType) || parameterType.equals(BOOLEAN)) { + passedType = parameterType; + } + else if (isDateTimeType(parameterType)) { + if (parameterType.equals(INTERVAL_DAY_TIME) || parameterType.equals(INTERVAL_YEAR_MONTH)) { + throw semanticException(INVALID_FUNCTION_ARGUMENT, parameter, "Invalid type of JSON path parameter: %s", parameterType.getDisplayName()); + } + passedType = parameterType; + } + else { + if (!typeCoercion.canCoerce(parameterType, VARCHAR)) { + throw semanticException(INVALID_FUNCTION_ARGUMENT, parameter, "Invalid type of JSON path parameter: %s", parameterType.getDisplayName()); + } + coerceType(parameter, parameterType, VARCHAR, "JSON path parameter"); + passedType = VARCHAR; + } + } + + types.put(parameterName, passedType); + fields.add(new RowType.Field(Optional.of(parameterName), passedType)); + } + + Type parametersRowType = JSON_NO_PARAMETERS_ROW_TYPE; + if (!pathParameters.isEmpty()) { + parametersRowType = RowType.from(fields.build()); + } + + // ANALYZE JSON PATH + Map typesMap = types.buildOrThrow(); + JsonPathAnalysis pathAnalysis = new JsonPathAnalyzer( + plannerContext.getMetadata(), + session, + createConstantAnalyzer(plannerContext, accessControl, session, ExpressionAnalyzer.this.parameters, WarningCollector.NOOP)) + .analyzeJsonPath(jsonPathInvocation.getJsonPath(), typesMap); + jsonPathAnalyses.put(NodeRef.of(node), pathAnalysis); + + return ImmutableList.of( + JSON_2016, // input expression + plannerContext.getTypeManager().getType(TypeId.of(JsonPath2016Type.NAME)), // parsed JSON path representation + parametersRowType); // passed parameters + } + + private ResolvedFunction getInputFunction(Type type, JsonFormat format, Node node) + { + QualifiedName name; + switch (format) { + case JSON: + if (UNKNOWN.equals(type) || isCharacterStringType(type)) { + name = QualifiedName.of(VARCHAR_TO_JSON); + } + else if (isStringType(type)) { + name = QualifiedName.of(VARBINARY_TO_JSON); + } + else { + throw semanticException(TYPE_MISMATCH, node, format("Cannot read input of type %s as JSON using formatting %s", type, format)); + } + break; + case UTF8: + name = QualifiedName.of(VARBINARY_UTF8_TO_JSON); + break; + case UTF16: + name = QualifiedName.of(VARBINARY_UTF16_TO_JSON); + break; + case UTF32: + name = QualifiedName.of(VARBINARY_UTF32_TO_JSON); + break; + default: + throw new UnsupportedOperationException("Unexpected format: " + format); + } + try { + return plannerContext.getMetadata().resolveFunction(session, name, fromTypes(type, BOOLEAN)); + } + catch (TrinoException e) { + throw new TrinoException(TYPE_MISMATCH, extractLocation(node), format("Cannot read input of type %s as JSON using formatting %s", type, format), e); + } + } + + private ResolvedFunction getOutputFunction(Type type, JsonFormat format, Node node) + { + QualifiedName name; + switch (format) { + case JSON: + if (isCharacterStringType(type)) { + name = QualifiedName.of(JSON_TO_VARCHAR); + } + else if (isStringType(type)) { + name = QualifiedName.of(JSON_TO_VARBINARY); + } + else { + throw semanticException(TYPE_MISMATCH, node, format("Cannot output JSON value as %s using formatting %s", type, format)); + } + break; + case UTF8: + if (!VARBINARY.equals(type)) { + throw semanticException(TYPE_MISMATCH, node, format("Cannot output JSON value as %s using formatting %s", type, format)); + } + name = QualifiedName.of(JSON_TO_VARBINARY_UTF8); + break; + case UTF16: + if (!VARBINARY.equals(type)) { + throw semanticException(TYPE_MISMATCH, node, format("Cannot output JSON value as %s using formatting %s", type, format)); + } + name = QualifiedName.of(JSON_TO_VARBINARY_UTF16); + break; + case UTF32: + if (!VARBINARY.equals(type)) { + throw semanticException(TYPE_MISMATCH, node, format("Cannot output JSON value as %s using formatting %s", type, format)); + } + name = QualifiedName.of(JSON_TO_VARBINARY_UTF32); + break; + default: + throw new UnsupportedOperationException("Unexpected format: " + format); + } + try { + return plannerContext.getMetadata().resolveFunction(session, name, fromTypes(JSON_2016, TINYINT, BOOLEAN)); + } + catch (TrinoException e) { + throw new TrinoException(TYPE_MISMATCH, extractLocation(node), format("Cannot output JSON value as %s using formatting %s", type, format), e); + } + } + private Type getOperator(StackableAstVisitorContext context, Expression node, OperatorType operatorType, Expression... arguments) { ImmutableList.Builder argumentTypes = ImmutableList.builder(); @@ -2889,6 +3302,9 @@ private static void updateAnalysis(Analysis analysis, ExpressionAnalyzer analyze analysis.setUndefinedLabels(analyzer.getUndefinedLabels()); analysis.setMeasureDefinitions(analyzer.getMeasureDefinitions()); analysis.setPatternAggregations(analyzer.getPatternAggregations()); + analysis.setJsonPathAnalyses(analyzer.getJsonPathAnalyses()); + analysis.setJsonInputFunctions(analyzer.getJsonInputFunctions()); + analysis.setJsonOutputFunctions(analyzer.getJsonOutputFunctions()); analysis.addPredicateCoercions(analyzer.getPredicateCoercions()); } @@ -2999,6 +3415,16 @@ private static boolean isExactNumericWithScaleZero(Type type) type instanceof DecimalType && ((DecimalType) type).getScale() == 0; } + public static boolean isStringType(Type type) + { + return isCharacterStringType(type) || VARBINARY.equals(type); + } + + public static boolean isCharacterStringType(Type type) + { + return type instanceof VarcharType || type instanceof CharType; + } + public static class LabelPrefixedReference { private final String label; diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/JsonPathAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/JsonPathAnalyzer.java new file mode 100644 index 000000000000..5e904f72890d --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/JsonPathAnalyzer.java @@ -0,0 +1,535 @@ +/* + * 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 io.trino.sql.analyzer; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.trino.Session; +import io.trino.metadata.BoundSignature; +import io.trino.metadata.Metadata; +import io.trino.metadata.OperatorNotFoundException; +import io.trino.spi.TrinoException; +import io.trino.spi.function.OperatorType; +import io.trino.spi.type.Type; +import io.trino.sql.jsonpath.PathNodeRef; +import io.trino.sql.jsonpath.PathParser; +import io.trino.sql.jsonpath.PathParser.Location; +import io.trino.sql.jsonpath.tree.AbsMethod; +import io.trino.sql.jsonpath.tree.ArithmeticBinary; +import io.trino.sql.jsonpath.tree.ArithmeticUnary; +import io.trino.sql.jsonpath.tree.ArrayAccessor; +import io.trino.sql.jsonpath.tree.ArrayAccessor.Subscript; +import io.trino.sql.jsonpath.tree.CeilingMethod; +import io.trino.sql.jsonpath.tree.ComparisonPredicate; +import io.trino.sql.jsonpath.tree.ConjunctionPredicate; +import io.trino.sql.jsonpath.tree.ContextVariable; +import io.trino.sql.jsonpath.tree.DatetimeMethod; +import io.trino.sql.jsonpath.tree.DisjunctionPredicate; +import io.trino.sql.jsonpath.tree.DoubleMethod; +import io.trino.sql.jsonpath.tree.ExistsPredicate; +import io.trino.sql.jsonpath.tree.Filter; +import io.trino.sql.jsonpath.tree.FloorMethod; +import io.trino.sql.jsonpath.tree.IsUnknownPredicate; +import io.trino.sql.jsonpath.tree.JsonNullLiteral; +import io.trino.sql.jsonpath.tree.JsonPath; +import io.trino.sql.jsonpath.tree.JsonPathTreeVisitor; +import io.trino.sql.jsonpath.tree.KeyValueMethod; +import io.trino.sql.jsonpath.tree.LastIndexVariable; +import io.trino.sql.jsonpath.tree.LikeRegexPredicate; +import io.trino.sql.jsonpath.tree.MemberAccessor; +import io.trino.sql.jsonpath.tree.NamedVariable; +import io.trino.sql.jsonpath.tree.NegationPredicate; +import io.trino.sql.jsonpath.tree.PathNode; +import io.trino.sql.jsonpath.tree.PredicateCurrentItemVariable; +import io.trino.sql.jsonpath.tree.SizeMethod; +import io.trino.sql.jsonpath.tree.SqlValueLiteral; +import io.trino.sql.jsonpath.tree.StartsWithPredicate; +import io.trino.sql.jsonpath.tree.TypeMethod; +import io.trino.sql.tree.Node; +import io.trino.sql.tree.QualifiedName; +import io.trino.sql.tree.StringLiteral; + +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkState; +import static io.trino.spi.StandardErrorCode.INVALID_PATH; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.function.OperatorType.NEGATION; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.VarcharType.createVarcharType; +import static io.trino.sql.analyzer.ExpressionAnalyzer.isCharacterStringType; +import static io.trino.sql.analyzer.ExpressionAnalyzer.isNumericType; +import static io.trino.sql.analyzer.ExpressionAnalyzer.isStringType; +import static io.trino.sql.analyzer.ExpressionTreeUtils.extractLocation; +import static io.trino.sql.analyzer.SemanticExceptions.semanticException; +import static io.trino.sql.analyzer.TypeSignatureProvider.fromTypes; +import static io.trino.sql.jsonpath.tree.ArithmeticUnary.Sign.PLUS; +import static io.trino.type.Json2016Type.JSON_2016; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class JsonPathAnalyzer +{ + // the type() method returns a textual description of type as determined by the SQL standard, of length lower or equal to 27 + private static final Type TYPE_METHOD_RESULT_TYPE = createVarcharType(27); + + private final Metadata metadata; + private final Session session; + private final ExpressionAnalyzer literalAnalyzer; + private final Map, Type> types = new LinkedHashMap<>(); + private final Set> jsonParameters = new LinkedHashSet<>(); + + public JsonPathAnalyzer(Metadata metadata, Session session, ExpressionAnalyzer literalAnalyzer) + { + this.metadata = requireNonNull(metadata, "metadata is null"); + this.session = requireNonNull(session, "session is null"); + this.literalAnalyzer = requireNonNull(literalAnalyzer, "literalAnalyzer is null"); + } + + public JsonPathAnalysis analyzeJsonPath(StringLiteral path, Map parameterTypes) + { + Location pathStart = extractLocation(path) + .map(location -> new Location(location.getLineNumber(), location.getColumnNumber())) + .orElseThrow(() -> new IllegalStateException("missing NodeLocation in path")); + PathNode root = new PathParser(pathStart).parseJsonPath(path.getValue()); + new Visitor(parameterTypes, path).process(root); + return new JsonPathAnalysis((JsonPath) root, types, jsonParameters); + } + + /** + * This visitor determines and validates output types of PathNodes, whenever they can be deduced and represented as SQL types. + * In some cases, the type of a PathNode can be determined without context. E.g., the `double()` method always returns DOUBLE. + * In some other cases, the type depends on child nodes. E.g. the return type of the `abs()` method is the same as input type. + * In some cases, the type cannot be represented as SQL type. E.g. the `keyValue()` method returns JSON objects. + * Some PathNodes, including accessors, return objects whose types might or might not be representable as SQL types, + * but that cannot be determined upfront. + */ + private class Visitor + extends JsonPathTreeVisitor + { + private final Map parameterTypes; + private final Node pathNode; + + public Visitor(Map parameterTypes, Node pathNode) + { + this.parameterTypes = ImmutableMap.copyOf(requireNonNull(parameterTypes, "parameterTypes is null")); + this.pathNode = requireNonNull(pathNode, "pathNode is null"); + } + + @Override + protected Type visitPathNode(PathNode node, Void context) + { + throw new UnsupportedOperationException("not supported JSON path node: " + node.getClass().getSimpleName()); + } + + @Override + protected Type visitAbsMethod(AbsMethod node, Void context) + { + Type sourceType = process(node.getBase()); + if (sourceType != null) { + Type resultType; + try { + resultType = metadata.resolveFunction(session, QualifiedName.of("abs"), fromTypes(sourceType)).getSignature().getReturnType(); + } + catch (TrinoException e) { + throw semanticException(INVALID_PATH, pathNode, e, "cannot perform JSON path abs() method with %s argument: %s", sourceType.getDisplayName(), e.getMessage()); + } + types.put(PathNodeRef.of(node), resultType); + return resultType; + } + + return null; + } + + @Override + protected Type visitArithmeticBinary(ArithmeticBinary node, Void context) + { + Type leftType = process(node.getLeft()); + Type rightType = process(node.getRight()); + if (leftType != null && rightType != null) { + BoundSignature signature; + try { + signature = metadata.resolveOperator(session, OperatorType.valueOf(node.getOperator().name()), ImmutableList.of(leftType, rightType)).getSignature(); + } + catch (OperatorNotFoundException e) { + throw semanticException(INVALID_PATH, pathNode, e, "invalid operand types (%s and %s) in JSON path arithmetic binary expression: %s", leftType.getDisplayName(), rightType.getDisplayName(), e.getMessage()); + } + Type resultType = signature.getReturnType(); + types.put(PathNodeRef.of(node), resultType); + return resultType; + } + + return null; + } + + @Override + protected Type visitArithmeticUnary(ArithmeticUnary node, Void context) + { + Type sourceType = process(node.getBase()); + if (sourceType != null) { + if (node.getSign() == PLUS) { + if (!isNumericType(sourceType)) { + throw semanticException(INVALID_PATH, pathNode, "Invalid operand type (%s) in JSON path arithmetic unary expression", sourceType.getDisplayName()); + } + types.put(PathNodeRef.of(node), sourceType); + return sourceType; + } + Type resultType; + try { + resultType = metadata.resolveOperator(session, NEGATION, ImmutableList.of(sourceType)).getSignature().getReturnType(); + } + catch (OperatorNotFoundException e) { + throw semanticException(INVALID_PATH, pathNode, e, "invalid operand type (%s) in JSON path arithmetic unary expression: %s", sourceType.getDisplayName(), e.getMessage()); + } + types.put(PathNodeRef.of(node), resultType); + return resultType; + } + + return null; + } + + @Override + protected Type visitArrayAccessor(ArrayAccessor node, Void context) + { + process(node.getBase()); + for (Subscript subscript : node.getSubscripts()) { + process(subscript.getFrom()); + subscript.getTo().ifPresent(this::process); + } + + return null; + } + + @Override + protected Type visitCeilingMethod(CeilingMethod node, Void context) + { + Type sourceType = process(node.getBase()); + if (sourceType != null) { + Type resultType; + try { + resultType = metadata.resolveFunction(session, QualifiedName.of("ceiling"), fromTypes(sourceType)).getSignature().getReturnType(); + } + catch (TrinoException e) { + throw semanticException(INVALID_PATH, pathNode, e, "cannot perform JSON path ceiling() method with %s argument: %s", sourceType.getDisplayName(), e.getMessage()); + } + types.put(PathNodeRef.of(node), resultType); + return resultType; + } + + return null; + } + + @Override + protected Type visitContextVariable(ContextVariable node, Void context) + { + return null; + } + + @Override + protected Type visitDatetimeMethod(DatetimeMethod node, Void context) + { + Type sourceType = process(node.getBase()); + if (sourceType != null && !isCharacterStringType(sourceType)) { + throw semanticException(INVALID_PATH, pathNode, "JSON path datetime() method requires character string argument (found %s)", sourceType.getDisplayName()); + } + // TODO process the format template, record the processed format, and deduce the returned type + throw semanticException(NOT_SUPPORTED, pathNode, "datetime method in JSON path is not yet supported"); + } + + @Override + protected Type visitDoubleMethod(DoubleMethod node, Void context) + { + Type sourceType = process(node.getBase()); + if (sourceType != null) { + if (!isStringType(sourceType) && !isNumericType(sourceType)) { + throw semanticException(INVALID_PATH, pathNode, "cannot perform JSON path double() method with %s argument", sourceType.getDisplayName()); + } + try { + metadata.getCoercion(session, sourceType, DOUBLE); + } + catch (OperatorNotFoundException e) { + throw semanticException(INVALID_PATH, pathNode, e, "cannot perform JSON path double() method with %s argument: %s", sourceType.getDisplayName(), e.getMessage()); + } + } + + types.put(PathNodeRef.of(node), DOUBLE); + return DOUBLE; + } + + @Override + protected Type visitFilter(Filter node, Void context) + { + Type sourceType = process(node.getBase()); + Type predicateType = process(node.getPredicate()); + + requireNonNull(predicateType, "missing type of predicate expression"); + checkState(predicateType.equals(BOOLEAN), "invalid type of predicate expression: " + predicateType.getDisplayName()); + + if (sourceType != null) { + types.put(PathNodeRef.of(node), sourceType); + return sourceType; + } + + return null; + } + + @Override + protected Type visitFloorMethod(FloorMethod node, Void context) + { + Type sourceType = process(node.getBase()); + if (sourceType != null) { + Type resultType; + try { + resultType = metadata.resolveFunction(session, QualifiedName.of("floor"), fromTypes(sourceType)).getSignature().getReturnType(); + } + catch (TrinoException e) { + throw semanticException(INVALID_PATH, pathNode, e, "cannot perform JSON path floor() method with %s argument: %s", sourceType.getDisplayName(), e.getMessage()); + } + types.put(PathNodeRef.of(node), resultType); + return resultType; + } + + return null; + } + + @Override + protected Type visitJsonNullLiteral(JsonNullLiteral node, Void context) + { + return null; + } + + @Override + protected Type visitJsonPath(JsonPath node, Void context) + { + Type type = process(node.getRoot()); + if (type != null) { + types.put(PathNodeRef.of(node), type); + } + return type; + } + + @Override + protected Type visitKeyValueMethod(KeyValueMethod node, Void context) + { + process(node.getBase()); + return null; + } + + @Override + protected Type visitLastIndexVariable(LastIndexVariable node, Void context) + { + types.put(PathNodeRef.of(node), INTEGER); + return INTEGER; + } + + @Override + protected Type visitMemberAccessor(MemberAccessor node, Void context) + { + process(node.getBase()); + return null; + } + + @Override + protected Type visitNamedVariable(NamedVariable node, Void context) + { + Type parameterType = parameterTypes.get(node.getName()); + if (parameterType == null) { + // This condition might be caused by the unintuitive semantics: + // identifiers in JSON path are case-sensitive, while non-delimited identifiers in SQL are upper-cased. + // Hence, a function call like JSON_VALUE(x, 'lax $var.floor()` PASSING 2.5 AS var) + // is an error, since the variable name is "var", and the passed parameter name is "VAR". + // We try to identify such situation and produce an explanatory message. + Optional similarName = parameterTypes.keySet().stream() + .filter(name -> name.equalsIgnoreCase(node.getName())) + .findFirst(); + if (similarName.isPresent()) { + throw semanticException(INVALID_PATH, pathNode, format("no value passed for parameter %s. Try quoting \"%s\" in the PASSING clause to match case", node.getName(), node.getName())); + } + throw semanticException(INVALID_PATH, pathNode, "no value passed for parameter " + node.getName()); + } + + if (parameterType.equals(JSON_2016)) { + jsonParameters.add(PathNodeRef.of(node)); + return null; + } + + // in case of a non-JSON named variable, the type cannot be recorded and used as the result type of the node + // this is because any incoming null value shall be transformed into a JSON null, which is out of the SQL type system. + // however, for any incoming non-null value, the type will be preserved. + return null; + } + + @Override + protected Type visitPredicateCurrentItemVariable(PredicateCurrentItemVariable node, Void context) + { + return null; + } + + @Override + protected Type visitSizeMethod(SizeMethod node, Void context) + { + process(node.getBase()); + types.put(PathNodeRef.of(node), INTEGER); + return INTEGER; + } + + @Override + protected Type visitSqlValueLiteral(SqlValueLiteral node, Void context) + { + Type type = literalAnalyzer.analyze(node.getValue(), Scope.create()); + types.put(PathNodeRef.of(node), type); + return type; + } + + @Override + protected Type visitTypeMethod(TypeMethod node, Void context) + { + process(node.getBase()); + Type type = TYPE_METHOD_RESULT_TYPE; + types.put(PathNodeRef.of(node), type); + return type; + } + + // predicate + + @Override + protected Type visitComparisonPredicate(ComparisonPredicate node, Void context) + { + process(node.getLeft()); + process(node.getRight()); + types.put(PathNodeRef.of(node), BOOLEAN); + return BOOLEAN; + } + + @Override + protected Type visitConjunctionPredicate(ConjunctionPredicate node, Void context) + { + Type leftType = process(node.getLeft()); + requireNonNull(leftType, "missing type of predicate expression"); + checkState(leftType.equals(BOOLEAN), "invalid type of predicate expression: " + leftType.getDisplayName()); + + Type rightType = process(node.getRight()); + requireNonNull(rightType, "missing type of predicate expression"); + checkState(rightType.equals(BOOLEAN), "invalid type of predicate expression: " + rightType.getDisplayName()); + + types.put(PathNodeRef.of(node), BOOLEAN); + return BOOLEAN; + } + + @Override + protected Type visitDisjunctionPredicate(DisjunctionPredicate node, Void context) + { + Type leftType = process(node.getLeft()); + requireNonNull(leftType, "missing type of predicate expression"); + checkState(leftType.equals(BOOLEAN), "invalid type of predicate expression: " + leftType.getDisplayName()); + + Type rightType = process(node.getRight()); + requireNonNull(rightType, "missing type of predicate expression"); + checkState(rightType.equals(BOOLEAN), "invalid type of predicate expression: " + rightType.getDisplayName()); + + types.put(PathNodeRef.of(node), BOOLEAN); + return BOOLEAN; + } + + @Override + protected Type visitExistsPredicate(ExistsPredicate node, Void context) + { + process(node.getPath()); + types.put(PathNodeRef.of(node), BOOLEAN); + return BOOLEAN; + } + + @Override + protected Type visitLikeRegexPredicate(LikeRegexPredicate node, Void context) + { + throw semanticException(NOT_SUPPORTED, pathNode, "like_regex predicate in JSON path is not yet supported"); + // TODO when like_regex is supported, this method should do the following: + // process(node.getPath()); + // types.put(PathNodeRef.of(node), BOOLEAN); + // return BOOLEAN; + } + + @Override + protected Type visitNegationPredicate(NegationPredicate node, Void context) + { + Type predicateType = process(node.getPredicate()); + requireNonNull(predicateType, "missing type of predicate expression"); + checkState(predicateType.equals(BOOLEAN), "invalid type of predicate expression: " + predicateType.getDisplayName()); + + types.put(PathNodeRef.of(node), BOOLEAN); + return BOOLEAN; + } + + @Override + protected Type visitStartsWithPredicate(StartsWithPredicate node, Void context) + { + process(node.getWhole()); + process(node.getInitial()); + types.put(PathNodeRef.of(node), BOOLEAN); + return BOOLEAN; + } + + @Override + protected Type visitIsUnknownPredicate(IsUnknownPredicate node, Void context) + { + Type predicateType = process(node.getPredicate()); + requireNonNull(predicateType, "missing type of predicate expression"); + checkState(predicateType.equals(BOOLEAN), "invalid type of predicate expression: " + predicateType.getDisplayName()); + + types.put(PathNodeRef.of(node), BOOLEAN); + return BOOLEAN; + } + } + + public static class JsonPathAnalysis + { + private final JsonPath path; + private final Map, Type> types; + private final Set> jsonParameters; + + public JsonPathAnalysis(JsonPath path, Map, Type> types, Set> jsonParameters) + { + this.path = requireNonNull(path, "path is null"); + this.types = ImmutableMap.copyOf(requireNonNull(types, "types is null")); + this.jsonParameters = ImmutableSet.copyOf(requireNonNull(jsonParameters, "jsonParameters is null")); + } + + public JsonPath getPath() + { + return path; + } + + public Type getType(PathNode pathNode) + { + return types.get(PathNodeRef.of(pathNode)); + } + + public Map, Type> getTypes() + { + return types; + } + + public Set> getJsonParameters() + { + return jsonParameters; + } + } +} diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/TranslationMap.java b/core/trino-main/src/main/java/io/trino/sql/planner/TranslationMap.java index 0ebadb8c0356..1ea52dbac8aa 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/TranslationMap.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/TranslationMap.java @@ -30,6 +30,9 @@ import io.trino.sql.tree.FunctionCall; import io.trino.sql.tree.GenericDataType; import io.trino.sql.tree.Identifier; +import io.trino.sql.tree.JsonExists; +import io.trino.sql.tree.JsonQuery; +import io.trino.sql.tree.JsonValue; import io.trino.sql.tree.LabelDereference; import io.trino.sql.tree.LambdaArgumentDeclaration; import io.trino.sql.tree.LambdaExpression; @@ -391,6 +394,24 @@ public Expression rewriteRowDataType(RowDataType node, Void context, ExpressionT return node; } + @Override + public Expression rewriteJsonExists(JsonExists node, Void context, ExpressionTreeRewriter treeRewriter) + { + throw new UnsupportedOperationException("JSON_EXISTS function is not yet supported"); + } + + @Override + public Expression rewriteJsonValue(JsonValue node, Void context, ExpressionTreeRewriter treeRewriter) + { + throw new UnsupportedOperationException("JSON_VALUE function is not yet supported"); + } + + @Override + public Expression rewriteJsonQuery(JsonQuery node, Void context, ExpressionTreeRewriter treeRewriter) + { + throw new UnsupportedOperationException("JSON_QUERY function is not yet supported"); + } + private Expression coerceIfNecessary(Expression original, Expression rewritten) { // Don't add a coercion for the top-level expression. That depends on the context the expression is used and it's the responsibility of the caller. diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index 977be2069e65..09b277a4a7e5 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -115,6 +115,9 @@ import io.trino.operator.TaskContext; import io.trino.operator.TrinoOperatorFactories; import io.trino.operator.index.IndexJoinLookupStats; +import io.trino.operator.scalar.json.JsonExistsFunction; +import io.trino.operator.scalar.json.JsonQueryFunction; +import io.trino.operator.scalar.json.JsonValueFunction; import io.trino.plugin.base.security.AllowAllSystemAccessControl; import io.trino.security.GroupProviderManager; import io.trino.server.PluginManager; @@ -188,6 +191,8 @@ import io.trino.transaction.TransactionManagerConfig; import io.trino.type.BlockTypeOperators; import io.trino.type.InternalTypeManager; +import io.trino.type.JsonPath2016Type; +import io.trino.type.TypeDeserializer; import io.trino.util.FinalizerService; import org.intellij.lang.annotations.Language; @@ -244,6 +249,7 @@ public class LocalQueryRunner private final InMemoryNodeManager nodeManager; private final BlockTypeOperators blockTypeOperators; private final PlannerContext plannerContext; + private final TypeRegistry typeRegistry; private final GlobalFunctionCatalog globalFunctionCatalog; private final FunctionManager functionManager; private final StatsCalculator statsCalculator; @@ -347,7 +353,7 @@ private LocalQueryRunner( this.nodePartitioningManager = new NodePartitioningManager(nodeScheduler, blockTypeOperators); BlockEncodingManager blockEncodingManager = new BlockEncodingManager(); - TypeRegistry typeRegistry = new TypeRegistry(typeOperators, featuresConfig); + typeRegistry = new TypeRegistry(typeOperators, featuresConfig); TypeManager typeManager = new InternalTypeManager(typeRegistry); InternalBlockEncodingSerde blockEncodingSerde = new InternalBlockEncodingSerde(blockEncodingManager, typeManager); @@ -361,6 +367,11 @@ private LocalQueryRunner( transactionManager, globalFunctionCatalog, typeManager); + globalFunctionCatalog.addFunctions(new InternalFunctionBundle( + new JsonExistsFunction(functionManager, metadata, typeManager), + new JsonValueFunction(functionManager, metadata, typeManager), + new JsonQueryFunction(functionManager, metadata, typeManager))); + typeRegistry.addType(new JsonPath2016Type(new TypeDeserializer(typeManager), blockEncodingSerde)); this.plannerContext = new PlannerContext(metadata, typeOperators, blockEncodingSerde, typeManager, functionManager); this.splitManager = new SplitManager(new QueryManagerConfig()); this.planFragmenter = new PlanFragmenter(metadata, functionManager, this.nodePartitioningManager, new QueryManagerConfig()); @@ -601,12 +612,22 @@ public AnalyzePropertyManager getAnalyzePropertyManager() return analyzePropertyManager; } + public TypeRegistry getTypeRegistry() + { + return typeRegistry; + } + @Override public TypeManager getTypeManager() { return plannerContext.getTypeManager(); } + public GlobalFunctionCatalog getGlobalFunctionCatalog() + { + return globalFunctionCatalog; + } + @Override public QueryExplainer getQueryExplainer() { diff --git a/core/trino-main/src/main/java/io/trino/type/Json2016Type.java b/core/trino-main/src/main/java/io/trino/type/Json2016Type.java new file mode 100644 index 000000000000..a6c8f0987ac9 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/type/Json2016Type.java @@ -0,0 +1,83 @@ +/* + * 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 io.trino.type; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.airlift.slice.Slice; +import io.trino.operator.scalar.json.JsonInputConversionError; +import io.trino.operator.scalar.json.JsonOutputConversionError; +import io.trino.spi.block.Block; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.AbstractVariableWidthType; +import io.trino.spi.type.StandardTypes; +import io.trino.spi.type.TypeSignature; + +import static io.airlift.slice.Slices.utf8Slice; + +public class Json2016Type + extends AbstractVariableWidthType +{ + public static final Json2016Type JSON_2016 = new Json2016Type(); + private static final ObjectMapper MAPPER = new ObjectMapper(); + + public Json2016Type() + { + super(new TypeSignature(StandardTypes.JSON_2016), JsonNode.class); + } + + @Override + public Object getObjectValue(ConnectorSession session, Block block, int position) + { + throw new UnsupportedOperationException(); + } + + @Override + public void appendTo(Block block, int position, BlockBuilder blockBuilder) + { + throw new UnsupportedOperationException(); + } + + @Override + public Object getObject(Block block, int position) + { + if (block.isNull(position)) { + return null; + } + + Slice bytes = block.getSlice(position, 0, block.getSliceLength(position)); + try { + return MAPPER.readTree(bytes.toStringUtf8()); + } + catch (JsonProcessingException e) { + throw new JsonInputConversionError(e); + } + } + + @Override + public void writeObject(BlockBuilder blockBuilder, Object value) + { + String json; + try { + json = MAPPER.writeValueAsString(value); + } + catch (JsonProcessingException e) { + throw new JsonOutputConversionError(e); + } + Slice bytes = utf8Slice(json); + blockBuilder.writeBytes(bytes, 0, bytes.length()).closeEntry(); + } +} diff --git a/core/trino-main/src/main/java/io/trino/type/JsonPath2016Type.java b/core/trino-main/src/main/java/io/trino/type/JsonPath2016Type.java new file mode 100644 index 000000000000..460d4b5422c1 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/type/JsonPath2016Type.java @@ -0,0 +1,86 @@ +/* + * 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 io.trino.type; + +import com.google.common.collect.ImmutableMap; +import io.airlift.json.JsonCodec; +import io.airlift.json.JsonCodecFactory; +import io.airlift.json.ObjectMapperProvider; +import io.airlift.slice.Slice; +import io.trino.block.BlockJsonSerde; +import io.trino.spi.block.Block; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.block.BlockEncodingSerde; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.AbstractVariableWidthType; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeSignature; +import io.trino.sql.jsonpath.tree.JsonPath; + +import static io.airlift.slice.Slices.utf8Slice; + +public class JsonPath2016Type + extends AbstractVariableWidthType +{ + public static final String NAME = "JsonPath2016"; + + private final JsonCodec jsonPathCodec; + + public JsonPath2016Type(TypeDeserializer typeDeserializer, BlockEncodingSerde blockEncodingSerde) + { + super(new TypeSignature(NAME), JsonPath.class); // TODO switch to IrJsonPath + this.jsonPathCodec = getCodec(typeDeserializer, blockEncodingSerde); + } + + @Override + public Object getObjectValue(ConnectorSession session, Block block, int position) + { + throw new UnsupportedOperationException(); + } + + @Override + public void appendTo(Block block, int position, BlockBuilder blockBuilder) + { + throw new UnsupportedOperationException(); + } + + @Override + public Object getObject(Block block, int position) + { + if (block.isNull(position)) { + return null; + } + + Slice bytes = block.getSlice(position, 0, block.getSliceLength(position)); + return jsonPathCodec.fromJson(bytes.toStringUtf8()); + } + + @Override + public void writeObject(BlockBuilder blockBuilder, Object value) + { + String json = jsonPathCodec.toJson((JsonPath) value); + Slice bytes = utf8Slice(json); + blockBuilder.writeBytes(bytes, 0, bytes.length()).closeEntry(); + } + + private static JsonCodec getCodec(TypeDeserializer typeDeserializer, BlockEncodingSerde blockEncodingSerde) + { + ObjectMapperProvider provider = new ObjectMapperProvider(); + provider.setJsonSerializers(ImmutableMap.of(Block.class, new BlockJsonSerde.Serializer(blockEncodingSerde))); + provider.setJsonDeserializers(ImmutableMap.of( + Type.class, typeDeserializer, + Block.class, new BlockJsonSerde.Deserializer(blockEncodingSerde))); + return new JsonCodecFactory(provider).jsonCodec(JsonPath.class); + } +} diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java index 2999e81b10b2..1aeaa201154d 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java @@ -94,6 +94,7 @@ import static io.trino.spi.StandardErrorCode.COLUMN_TYPE_UNKNOWN; import static io.trino.spi.StandardErrorCode.DUPLICATE_COLUMN_NAME; import static io.trino.spi.StandardErrorCode.DUPLICATE_NAMED_QUERY; +import static io.trino.spi.StandardErrorCode.DUPLICATE_PARAMETER_NAME; import static io.trino.spi.StandardErrorCode.DUPLICATE_PROPERTY; import static io.trino.spi.StandardErrorCode.DUPLICATE_WINDOW_NAME; import static io.trino.spi.StandardErrorCode.EXPRESSION_NOT_AGGREGATE; @@ -5296,6 +5297,348 @@ public void testAnalyzeMaterializedViewWithAccessControl() .hasMessage("Access Denied: Cannot select from columns [a, b] in table or view tpch.s1.fresh_materialized_view"); } + @Test + public void testJsonContextItemType() + { + analyze("SELECT JSON_EXISTS(json_column, 'lax $.abs()') FROM (VALUES '-1', 'ala') t(json_column)"); + analyze("SELECT JSON_EXISTS(json_column, 'lax $.abs()') FROM (VALUES X'65683F', X'65683E') t(json_column)"); + + assertFails("SELECT JSON_EXISTS(json_column, 'lax $.abs()') FROM (VALUES -1, -2) t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:20: Cannot read input of type integer as JSON using formatting JSON"); + } + + @Test + public void testJsonContextItemFormat() + { + // implicit FORMAT JSON + analyze("SELECT JSON_EXISTS(json_column, 'lax $.abs()') FROM (VALUES '-1', 'ala') t(json_column)"); + analyze("SELECT JSON_EXISTS(json_column, 'lax $.abs()') FROM (VALUES X'65683F', X'65683E') t(json_column)"); + + // explicit input format + analyze("SELECT JSON_EXISTS(json_column FORMAT JSON, 'lax $.abs()') FROM (VALUES '-1', 'ala') t(json_column)"); + analyze("SELECT JSON_EXISTS(json_column FORMAT JSON ENCODING UTF8, 'lax $.abs()') FROM (VALUES X'1A', X'2B') t(json_column)"); + analyze("SELECT JSON_EXISTS(json_column FORMAT JSON ENCODING UTF16, 'lax $.abs()') FROM (VALUES X'1A', X'2B') t(json_column)"); + analyze("SELECT JSON_EXISTS(json_column FORMAT JSON ENCODING UTF32, 'lax $.abs()') FROM (VALUES X'1A', X'2B') t(json_column)"); + + // incorrect format: ENCODING specified for character string input + assertFails("SELECT JSON_EXISTS(json_column FORMAT JSON ENCODING UTF8, 'lax $.abs()') FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:20: Cannot read input of type varchar(3) as JSON using formatting JSON ENCODING UTF8"); + } + + @Test + public void testJsonPathParameterNames() + { + analyze("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING " + + " 1 AS parameter_1, " + + " 'x' AS parameter_2, " + + " true AS parameter_3) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + assertFails("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING " + + " 1 AS parameter_1, " + + " 'x' AS parameter_2, " + + " true AS parameter_1) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(DUPLICATE_PARAMETER_NAME) + .hasMessage("line 1:309: PARAMETER_1 JSON path parameter is specified more than once"); + } + + @Test + public void testCaseSensitiveNames() + { + // JSON path variable names are case-sensitive. Unquoted parameter names in the PASSING clause are upper-cased. + analyze("SELECT JSON_EXISTS(json_column, 'lax $some_name' PASSING 1 AS \"some_name\") FROM (VALUES '-1', 'ala') t(json_column)"); + analyze("SELECT JSON_EXISTS(json_column, 'lax $SOME_NAME' PASSING 1 AS some_name) FROM (VALUES '-1', 'ala') t(json_column)"); + + // no matching parameter, but similar parameter found with different case. provide a hint in the error message + assertFails("SELECT JSON_EXISTS(json_column, 'lax $some_name' PASSING 1 AS some_name) FROM (VALUES '-1', 'ala') t(json_column)") + .hasMessage("line 1:33: no value passed for parameter some_name. Try quoting \"some_name\" in the PASSING clause to match case"); + + assertFails("SELECT JSON_EXISTS(json_column, 'lax $some_NAME' PASSING 1 AS some_name) FROM (VALUES '-1', 'ala') t(json_column)") + .hasMessage("line 1:33: no value passed for parameter some_NAME. Try quoting \"some_NAME\" in the PASSING clause to match case"); + + // no matching parameter, and it is not the issue with case sensitivity. no hint in the error message + assertFails("SELECT JSON_EXISTS(json_column, 'lax $some_name' PASSING 1 AS some_other_name) FROM (VALUES '-1', 'ala') t(json_column)") + .hasMessage("line 1:33: no value passed for parameter some_name"); + } + + @Test + public void testJsonPathParameterFormats() + { + analyze("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING 'x' FORMAT JSON AS parameter_1) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + analyze("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING X'65683F' FORMAT JSON ENCODING UTF8 AS parameter_1) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + assertFails("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING 1 FORMAT JSON AS parameter_1) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:110: Cannot read input of type integer as JSON using formatting JSON"); + + assertFails("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING 1 FORMAT JSON ENCODING UTF8 AS parameter_1) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:110: Cannot read input of type integer as JSON using formatting JSON ENCODING UTF8"); + + // FORMAT JSON as the parameter format option is the same as the output format of the JSON_QUERY call + analyze("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING JSON_QUERY(json_column, 'lax $.abs()' RETURNING varchar FORMAT JSON) FORMAT JSON AS parameter_1) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + // FORMAT JSON as the parameter format option is different than the output format of the JSON_QUERY call + analyze("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING JSON_QUERY(json_column, 'lax $.abs()' RETURNING varbinary FORMAT JSON) FORMAT JSON ENCODING UTF8 AS parameter_1) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + // the parameter is a JSON_QUERY call, so the format option FORMAT JSON is implicit for the parameter + analyze("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING JSON_QUERY(json_column, 'lax $.abs()' RETURNING varchar FORMAT JSON) AS parameter_1) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + } + + @Test + public void testJsonPathParameterTypes() + { + assertFails("SELECT JSON_EXISTS( " + + " json_column, " + + " 'lax $.abs()' PASSING INTERVAL '2' DAY AS parameter_1) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(INVALID_FUNCTION_ARGUMENT) + .hasMessage("line 1:110: Invalid type of JSON path parameter: interval day to second"); + } + + @Test + public void testJsonValueReturnedType() + { + analyze("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING char(30)) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + analyze("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.size()'" + + " RETURNING bigint) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + assertFails("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING tdigest) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:8: Invalid return type of function JSON_VALUE: tdigest"); + + assertFails("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING some_type(10)) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:8: Unknown type: some_type(10)"); + } + + @Test + public void testJsonValueDefaultValues() + { + // default value has the same type as the declared returned type + analyze("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.double()'" + + " RETURNING double" + + " DEFAULT 1e0 ON EMPTY) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + // default value can be coerced to the declared returned type + analyze("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.double()'" + + " RETURNING double" + + " DEFAULT 1.0 ON EMPTY) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + assertFails("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.double()'" + + " RETURNING double" + + " DEFAULT 'text' ON EMPTY) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:149: Function JSON_VALUE default ON EMPTY result must evaluate to a double (actual: varchar(4))"); + + // default value has the same type as the declared returned type + analyze("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.double()'" + + " RETURNING double" + + " DEFAULT 1e0 ON ERROR) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + // default value can be coerced to the declared returned type + analyze("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.double()'" + + " RETURNING double" + + " DEFAULT 1.0 ON ERROR) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + assertFails("SELECT JSON_VALUE( " + + " json_column, " + + " 'lax $.double()'" + + " RETURNING double" + + " DEFAULT 'text' ON ERROR) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:149: Function JSON_VALUE default ON ERROR result must evaluate to a double (actual: varchar(4))"); + } + + @Test + public void testJsonQueryOutputTypeAndFormat() + { + analyze("SELECT JSON_QUERY( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING varchar) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + analyze("SELECT JSON_QUERY( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING varchar FORMAT JSON) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + analyze("SELECT JSON_QUERY( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING char(5) FORMAT JSON) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + analyze("SELECT JSON_QUERY( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING varbinary FORMAT JSON ENCODING UTF8) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + assertFails("SELECT JSON_QUERY( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING some_type(10)) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:8: Unknown type: some_type(10)"); + + assertFails("SELECT JSON_QUERY( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING double) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:8: Cannot output JSON value as double using formatting JSON"); + + assertFails("SELECT JSON_QUERY( " + + " json_column, " + + " 'lax $.type()'" + + " RETURNING varchar FORMAT JSON ENCODING UTF8) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(TYPE_MISMATCH) + .hasMessage("line 1:8: Cannot output JSON value as varchar using formatting JSON ENCODING UTF8"); + } + + @Test + public void testJsonQueryQuotesBehavior() + { + analyze("SELECT JSON_QUERY( " + + " json_column, " + + " 'lax $.type()'" + + " OMIT QUOTES ON SCALAR STRING) " + + " FROM (VALUES '-1', 'ala') t(json_column)"); + + assertFails("SELECT JSON_QUERY( " + + " json_column, " + + " 'lax $.type()' " + + " WITH ARRAY WRAPPER " + + " OMIT QUOTES ON SCALAR STRING) " + + " FROM (VALUES '-1', 'ala') t(json_column)") + .hasErrorCode(INVALID_FUNCTION_ARGUMENT) + .hasMessage("line 1:8: OMIT QUOTES behavior specified with WITH UNCONDITIONAL ARRAY WRAPPER behavior"); + } + + @Test + public void testJsonExistsInAggregationContext() + { + analyze("SELECT JSON_EXISTS('-5', 'lax $.abs()') FROM (VALUES '-1', '-2') t(a) GROUP BY a"); + analyze("SELECT JSON_EXISTS(a, 'lax $.abs()') FROM (VALUES '-1', '-2') t(a) GROUP BY a"); + analyze("SELECT JSON_EXISTS(a, 'lax $.abs() + $some_number' PASSING b AS \"some_number\") FROM (VALUES ('-1', 10, 100), ('-2', 20, 200)) t(a, b, c) GROUP BY a, b"); + + assertFails("SELECT JSON_EXISTS(c, 'lax $.abs() + $some_number' PASSING b AS \"some_number\") FROM (VALUES ('-1', 10, '100'), ('-2', 20, '200')) t(a, b, c) GROUP BY a, b") + .hasErrorCode(EXPRESSION_NOT_AGGREGATE) + .hasMessage("line 1:8: 'JSON_EXISTS(c FORMAT JSON, 'lax $.abs() + $some_number' PASSING b AS \"some_number\" FALSE ON ERROR)' must be an aggregate expression or appear in GROUP BY clause"); + + assertFails("SELECT JSON_EXISTS(b, 'lax $.abs() + $some_number' PASSING c AS \"some_number\") FROM (VALUES (-1, '10', 100), (-2, '20', 200)) t(a, b, c) GROUP BY a, b") + .hasErrorCode(EXPRESSION_NOT_AGGREGATE) + .hasMessage("line 1:8: 'JSON_EXISTS(b FORMAT JSON, 'lax $.abs() + $some_number' PASSING c AS \"some_number\" FALSE ON ERROR)' must be an aggregate expression or appear in GROUP BY clause"); + } + + @Test + public void testJsonValueInAggregationContext() + { + analyze("SELECT JSON_VALUE('-5', 'lax $.abs()') FROM (VALUES '-1', '-2') t(a) GROUP BY a"); + analyze("SELECT JSON_VALUE(a, 'lax $.abs()') FROM (VALUES '-1', '-2') t(a) GROUP BY a"); + analyze("SELECT JSON_VALUE(a, 'lax $.abs() + $some_number' PASSING b AS \"some_number\") FROM (VALUES ('-1', 10, 100), ('-2', 20, 200)) t(a, b, c) GROUP BY a, b"); + analyze("SELECT JSON_VALUE(a, 'lax $.abs() + $some_number' PASSING b AS \"some_number\" DEFAULT lower(b) ON EMPTY DEFAULT upper(b) ON ERROR) FROM (VALUES ('-1', '10', 100), ('-2', '20', 200)) t(a, b, c) GROUP BY a, b"); + + assertFails("SELECT JSON_VALUE(c, 'lax $.abs() + $some_number' PASSING b AS \"some_number\") FROM (VALUES ('-1', 10, '100'), ('-2', 20, '200')) t(a, b, c) GROUP BY a, b") + .hasErrorCode(EXPRESSION_NOT_AGGREGATE) + .hasMessage("line 1:8: 'JSON_VALUE(c FORMAT JSON, 'lax $.abs() + $some_number' PASSING b AS \"some_number\" NULL ON EMPTY NULL ON ERROR)' must be an aggregate expression or appear in GROUP BY clause"); + + assertFails("SELECT JSON_VALUE(b, 'lax $.abs() + $some_number' PASSING c AS \"some_number\") FROM (VALUES (-1, '10', 100), (-2, '20', 200)) t(a, b, c) GROUP BY a, b") + .hasErrorCode(EXPRESSION_NOT_AGGREGATE) + .hasMessage("line 1:8: 'JSON_VALUE(b FORMAT JSON, 'lax $.abs() + $some_number' PASSING c AS \"some_number\" NULL ON EMPTY NULL ON ERROR)' must be an aggregate expression or appear in GROUP BY clause"); + + assertFails("SELECT JSON_VALUE(b, 'lax $.abs() + $some_number' PASSING b AS \"some_number\" DEFAULT c ON EMPTY) FROM (VALUES (-1, '10', '100'), (-2, '20', '200')) t(a, b, c) GROUP BY a, b") + .hasErrorCode(EXPRESSION_NOT_AGGREGATE) + .hasMessage("line 1:8: 'JSON_VALUE(b FORMAT JSON, 'lax $.abs() + $some_number' PASSING b AS \"some_number\" DEFAULT c ON EMPTY NULL ON ERROR)' must be an aggregate expression or appear in GROUP BY clause"); + + assertFails("SELECT JSON_VALUE(b, 'lax $.abs() + $some_number' PASSING b AS \"some_number\" DEFAULT c ON ERROR) FROM (VALUES (-1, '10', '100'), (-2, '20', '200')) t(a, b, c) GROUP BY a, b") + .hasErrorCode(EXPRESSION_NOT_AGGREGATE) + .hasMessage("line 1:8: 'JSON_VALUE(b FORMAT JSON, 'lax $.abs() + $some_number' PASSING b AS \"some_number\" NULL ON EMPTY DEFAULT c ON ERROR)' must be an aggregate expression or appear in GROUP BY clause"); + } + + @Test + public void testJsonQueryInAggregationContext() + { + analyze("SELECT JSON_QUERY('-5', 'lax $.abs()') FROM (VALUES '-1', '-2') t(a) GROUP BY a"); + analyze("SELECT JSON_QUERY(a, 'lax $.abs()') FROM (VALUES '-1', '-2') t(a) GROUP BY a"); + analyze("SELECT JSON_QUERY(a, 'lax $.abs() + $some_number' PASSING b AS \"some_number\") FROM (VALUES ('-1', 10, 100), ('-2', 20, 200)) t(a, b, c) GROUP BY a, b"); + + assertFails("SELECT JSON_QUERY(c, 'lax $.abs() + $some_number' PASSING b AS \"some_number\") FROM (VALUES ('-1', 10, '100'), ('-2', 20, '200')) t(a, b, c) GROUP BY a, b") + .hasErrorCode(EXPRESSION_NOT_AGGREGATE) + .hasMessage("line 1:8: 'JSON_QUERY(c FORMAT JSON, 'lax $.abs() + $some_number' PASSING b AS \"some_number\" WITHOUT ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)' must be an aggregate expression or appear in GROUP BY clause"); + + assertFails("SELECT JSON_QUERY(b, 'lax $.abs() + $some_number' PASSING c AS \"some_number\") FROM (VALUES (-1, '10', 100), (-2, '20', 200)) t(a, b, c) GROUP BY a, b") + .hasErrorCode(EXPRESSION_NOT_AGGREGATE) + .hasMessage("line 1:8: 'JSON_QUERY(b FORMAT JSON, 'lax $.abs() + $some_number' PASSING c AS \"some_number\" WITHOUT ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)' must be an aggregate expression or appear in GROUP BY clause"); + } + @BeforeClass public void setup() { diff --git a/core/trino-parser/src/main/antlr4/io/trino/jsonpath/JsonPath.g4 b/core/trino-parser/src/main/antlr4/io/trino/jsonpath/JsonPath.g4 new file mode 100644 index 000000000000..90dcd5e6cf1c --- /dev/null +++ b/core/trino-parser/src/main/antlr4/io/trino/jsonpath/JsonPath.g4 @@ -0,0 +1,200 @@ +/* + * 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. + */ + +grammar JsonPath; + +tokens { + DELIMITER +} + +path + : pathMode pathExpression EOF + ; + +pathMode + : LAX + | STRICT + ; + +pathExpression + : accessorExpression #expressionDefault + | sign=('+' | '-') pathExpression #signedUnary + | left=pathExpression operator=('*' | '/' | '%') right=pathExpression #binary + | left=pathExpression operator=('+' | '-') right=pathExpression #binary + ; + +accessorExpression + : pathPrimary #accessorExpressionDefault + | accessorExpression '.' identifier #memberAccessor + | accessorExpression '.' stringLiteral #memberAccessor + | accessorExpression '.' '*' #wildcardMemberAccessor + | accessorExpression '[' subscript (',' subscript)* ']' #arrayAccessor + | accessorExpression '[' '*' ']' #wildcardArrayAccessor + | accessorExpression '?' '(' predicate ')' #filter + | accessorExpression '.' TYPE '(' ')' #typeMethod + | accessorExpression '.' SIZE '(' ')' #sizeMethod + | accessorExpression '.' DOUBLE '(' ')' #doubleMethod + | accessorExpression '.' CEILING '(' ')' #ceilingMethod + | accessorExpression '.' FLOOR '(' ')' #floorMethod + | accessorExpression '.' ABS '(' ')' #absMethod + | accessorExpression '.' DATETIME '(' stringLiteral? ')' #datetimeMethod + | accessorExpression '.' KEYVALUE '(' ')' #keyValueMethod + ; + +identifier + : IDENTIFIER + | nonReserved + ; + +subscript + : singleton=pathExpression + | from=pathExpression TO to=pathExpression + ; + +pathPrimary + : literal #literalPrimary + | variable #variablePrimary + | '(' pathExpression ')' #parenthesizedPath + ; + +literal + : numericLiteral + | stringLiteral + | nullLiteral + | booleanLiteral + ; + +numericLiteral + : MINUS? DECIMAL_VALUE #decimalLiteral + | MINUS? DOUBLE_VALUE #doubleLiteral + | MINUS? INTEGER_VALUE #integerLiteral + ; + +stringLiteral + : STRING // add unicode (like SqlBase.g4), add quoting in single quotes (') + ; + +nullLiteral + : NULL + ; + +booleanLiteral + : TRUE | FALSE + ; + +variable + : '$' #contextVariable + | NAMED_VARIABLE #namedVariable + | LAST #lastIndexVariable + | '@' #predicateCurrentItemVariable + ; + +// the following part is dedicated to JSON path predicate +predicate + : predicatePrimary #predicateDefault + | '!' delimitedPredicate #negationPredicate + | left=predicate '&&' right=predicate #conjunctionPredicate + | left=predicate '||' right=predicate #disjunctionPredicate + ; + +predicatePrimary + : delimitedPredicate #predicatePrimaryDefault + | left=pathExpression comparisonOperator right=pathExpression #comparisonPredicate + | base=pathExpression LIKE_REGEX pattern=stringLiteral ( FLAG flag=stringLiteral )? #likeRegexPredicate + | whole=pathExpression STARTS WITH (string=stringLiteral | NAMED_VARIABLE) #startsWithPredicate + | '(' predicate ')' IS UNKNOWN #isUnknownPredicate + ; + +delimitedPredicate + : EXISTS '(' pathExpression ')' #existsPredicate + | '(' predicate ')' #parenthesizedPredicate + ; + +comparisonOperator + : '==' | '<>' | '!=' | '<' | '>' | '<=' | '>=' + ; + +// there shall be no reserved words in JSON path +nonReserved + : ABS | CEILING | DATETIME | DOUBLE | EXISTS | FALSE | FLAG | FLOOR | IS | KEYVALUE | LAST | LAX | LIKE_REGEX | MINUS | NULL | SIZE | STARTS | STRICT | TO | TRUE | TYPE | UNKNOWN | WITH + ; + +ABS: 'abs'; +CEILING: 'ceiling'; +DATETIME: 'datetime'; +DOUBLE: 'double'; +EXISTS: 'exists'; +FALSE: 'false'; +FLAG: 'flag'; +FLOOR: 'floor'; +IS: 'is'; +KEYVALUE: 'keyvalue'; +LAST: 'last'; +LAX: 'lax'; +LIKE_REGEX: 'like_regex'; +MINUS: '-'; +NULL: 'null'; +SIZE: 'size'; +STARTS: 'starts'; +STRICT: 'strict'; +TO: 'to'; +TRUE: 'true'; +TYPE: 'type'; +UNKNOWN: 'unknown'; +WITH: 'with'; + +DECIMAL_VALUE + : DIGIT+ '.' DIGIT* + | '.' DIGIT+ + ; + +DOUBLE_VALUE + : DIGIT+ ('.' DIGIT*)? EXPONENT + | '.' DIGIT+ EXPONENT + ; + +INTEGER_VALUE + : DIGIT+ + ; + +STRING + : '"' ( ~'"' | '""' )* '"' + ; + +IDENTIFIER + : (LETTER | '_') (LETTER | DIGIT | '_')* + ; + +NAMED_VARIABLE + : '$' IDENTIFIER + ; + +fragment EXPONENT + : ('E' | 'e') [+-]? DIGIT+ + ; + +fragment DIGIT + : [0-9] + ; + +fragment LETTER + : [a-z] | [A-Z] + ; + +WS + : [ \r\n\t]+ -> channel(HIDDEN) + ; + +// Catch-all for anything we can't recognize. +UNRECOGNIZED: .; diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathNodeRef.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathNodeRef.java new file mode 100644 index 000000000000..0b56d09d8e42 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathNodeRef.java @@ -0,0 +1,68 @@ +/* + * 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 io.trino.sql.jsonpath; + +import io.trino.sql.jsonpath.tree.PathNode; + +import static java.lang.String.format; +import static java.lang.System.identityHashCode; +import static java.util.Objects.requireNonNull; + +public final class PathNodeRef +{ + public static PathNodeRef of(T pathNode) + { + return new PathNodeRef<>(pathNode); + } + + private final T pathNode; + + private PathNodeRef(T pathNode) + { + this.pathNode = requireNonNull(pathNode, "pathNode is null"); + } + + public T getNode() + { + return pathNode; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PathNodeRef other = (PathNodeRef) o; + return pathNode == other.pathNode; + } + + @Override + public int hashCode() + { + return identityHashCode(pathNode); + } + + @Override + public String toString() + { + return format( + "@%s: %s", + Integer.toHexString(identityHashCode(pathNode)), + pathNode); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathParser.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathParser.java new file mode 100644 index 000000000000..2e54556165d6 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathParser.java @@ -0,0 +1,167 @@ +/* + * 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 io.trino.sql.jsonpath; + +import io.trino.jsonpath.JsonPathBaseListener; +import io.trino.jsonpath.JsonPathLexer; +import io.trino.jsonpath.JsonPathParser; +import io.trino.sql.jsonpath.tree.PathNode; +import io.trino.sql.parser.ParsingException; +import org.antlr.v4.runtime.BaseErrorListener; +import org.antlr.v4.runtime.CharStreams; +import org.antlr.v4.runtime.CommonToken; +import org.antlr.v4.runtime.CommonTokenStream; +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.RecognitionException; +import org.antlr.v4.runtime.Recognizer; +import org.antlr.v4.runtime.Token; +import org.antlr.v4.runtime.atn.PredictionMode; +import org.antlr.v4.runtime.misc.Pair; +import org.antlr.v4.runtime.misc.ParseCancellationException; +import org.antlr.v4.runtime.tree.TerminalNode; + +import java.util.Arrays; +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public final class PathParser +{ + private final BaseErrorListener errorListener; + + public PathParser(Location startLocation) + { + requireNonNull(startLocation, "startLocation is null"); + + int pathStartLine = startLocation.line; + int pathStartColumn = startLocation.column; + this.errorListener = new BaseErrorListener() + { + @Override + public void syntaxError(Recognizer recognizer, Object offendingSymbol, int line, int charPositionInLine, String message, RecognitionException e) + { + // The line and charPositionInLine correspond to the character within the string literal with JSON path expression. + // Line and offset in error returned to the user should be computed based on the beginning of the whole query text. + // We re-position the exception relatively to the start of the path expression within the query. + int lineInQuery = pathStartLine - 1 + line; + int columnInQuery = line == 1 ? pathStartColumn + 1 + charPositionInLine : charPositionInLine + 1; + throw new ParsingException(message, e, lineInQuery, columnInQuery); + } + }; + } + + public PathNode parseJsonPath(String path) + { + try { + // according to the SQL specification, the path language is case-sensitive in both identifiers and key words + JsonPathLexer lexer = new JsonPathLexer(CharStreams.fromString(path)); + CommonTokenStream tokenStream = new CommonTokenStream(lexer); + JsonPathParser parser = new JsonPathParser(tokenStream); + + parser.addParseListener(new PostProcessor(Arrays.asList(parser.getRuleNames()), parser)); + + lexer.removeErrorListeners(); + lexer.addErrorListener(errorListener); + + parser.removeErrorListeners(); + parser.addErrorListener(errorListener); + + ParserRuleContext tree; + try { + // first, try parsing with potentially faster SLL mode + parser.getInterpreter().setPredictionMode(PredictionMode.SLL); + tree = parser.path(); + } + catch (ParseCancellationException ex) { + // if we fail, parse with LL mode + tokenStream.seek(0); // rewind input stream + parser.reset(); + + parser.getInterpreter().setPredictionMode(PredictionMode.LL); + tree = parser.path(); + } + + return new PathTreeBuilder().visit(tree); + } + catch (StackOverflowError e) { + throw new ParsingException("stack overflow while parsing JSON path"); + } + } + + private static class PostProcessor + extends JsonPathBaseListener + { + private final List ruleNames; + private final JsonPathParser parser; + + public PostProcessor(List ruleNames, JsonPathParser parser) + { + this.ruleNames = ruleNames; + this.parser = parser; + } + + @Override + public void exitNonReserved(JsonPathParser.NonReservedContext context) + { + // only a terminal can be replaced during rule exit event handling. Make sure that the nonReserved item is a token + if (!(context.getChild(0) instanceof TerminalNode)) { + int rule = ((ParserRuleContext) context.getChild(0)).getRuleIndex(); + throw new AssertionError("nonReserved can only contain tokens. Found nested rule: " + ruleNames.get(rule)); + } + + // replace nonReserved keyword with IDENTIFIER token + context.getParent().removeLastChild(); + + Token token = (Token) context.getChild(0).getPayload(); + Token newToken = new CommonToken( + new Pair<>(token.getTokenSource(), token.getInputStream()), + JsonPathLexer.IDENTIFIER, + token.getChannel(), + token.getStartIndex(), + token.getStopIndex()); + + context.getParent().addChild(parser.createTerminalNode(context.getParent(), newToken)); + } + } + + public static class Location + { + private final int line; + private final int column; + + public Location(int line, int column) + { + if (line < 1) { + throw new IllegalArgumentException("line must be at least 1"); + } + + if (column < 0) { + throw new IllegalArgumentException("column must be at least 0"); + } + + this.line = line; + this.column = column; + } + + public int getLine() + { + return line; + } + + public int getColumn() + { + return column; + } + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathTreeBuilder.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathTreeBuilder.java new file mode 100644 index 000000000000..aa8475276fa7 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathTreeBuilder.java @@ -0,0 +1,421 @@ +/* + * 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 io.trino.sql.jsonpath; + +import com.google.common.collect.ImmutableList; +import io.trino.jsonpath.JsonPathBaseVisitor; +import io.trino.jsonpath.JsonPathParser; +import io.trino.sql.jsonpath.tree.AbsMethod; +import io.trino.sql.jsonpath.tree.ArithmeticBinary; +import io.trino.sql.jsonpath.tree.ArithmeticBinary.Operator; +import io.trino.sql.jsonpath.tree.ArithmeticUnary; +import io.trino.sql.jsonpath.tree.ArithmeticUnary.Sign; +import io.trino.sql.jsonpath.tree.ArrayAccessor; +import io.trino.sql.jsonpath.tree.CeilingMethod; +import io.trino.sql.jsonpath.tree.ComparisonPredicate; +import io.trino.sql.jsonpath.tree.ConjunctionPredicate; +import io.trino.sql.jsonpath.tree.ContextVariable; +import io.trino.sql.jsonpath.tree.DatetimeMethod; +import io.trino.sql.jsonpath.tree.DisjunctionPredicate; +import io.trino.sql.jsonpath.tree.DoubleMethod; +import io.trino.sql.jsonpath.tree.ExistsPredicate; +import io.trino.sql.jsonpath.tree.Filter; +import io.trino.sql.jsonpath.tree.FloorMethod; +import io.trino.sql.jsonpath.tree.IsUnknownPredicate; +import io.trino.sql.jsonpath.tree.JsonPath; +import io.trino.sql.jsonpath.tree.KeyValueMethod; +import io.trino.sql.jsonpath.tree.LastIndexVariable; +import io.trino.sql.jsonpath.tree.LikeRegexPredicate; +import io.trino.sql.jsonpath.tree.MemberAccessor; +import io.trino.sql.jsonpath.tree.NamedVariable; +import io.trino.sql.jsonpath.tree.NegationPredicate; +import io.trino.sql.jsonpath.tree.PathNode; +import io.trino.sql.jsonpath.tree.Predicate; +import io.trino.sql.jsonpath.tree.PredicateCurrentItemVariable; +import io.trino.sql.jsonpath.tree.SizeMethod; +import io.trino.sql.jsonpath.tree.SqlValueLiteral; +import io.trino.sql.jsonpath.tree.StartsWithPredicate; +import io.trino.sql.jsonpath.tree.TypeMethod; +import io.trino.sql.tree.BooleanLiteral; +import io.trino.sql.tree.DecimalLiteral; +import io.trino.sql.tree.DoubleLiteral; +import io.trino.sql.tree.LongLiteral; +import io.trino.sql.tree.StringLiteral; +import org.antlr.v4.runtime.tree.TerminalNode; + +import java.util.Optional; + +import static io.trino.sql.jsonpath.tree.JsonNullLiteral.JSON_NULL; + +public class PathTreeBuilder + extends JsonPathBaseVisitor +{ + @Override + public PathNode visitPath(JsonPathParser.PathContext context) + { + boolean lax = context.pathMode().LAX() != null; + PathNode path = visit(context.pathExpression()); + return new JsonPath(lax, path); + } + + @Override + public PathNode visitDecimalLiteral(JsonPathParser.DecimalLiteralContext context) + { + return new SqlValueLiteral(new DecimalLiteral(context.getText())); + } + + @Override + public PathNode visitDoubleLiteral(JsonPathParser.DoubleLiteralContext context) + { + return new SqlValueLiteral(new DoubleLiteral(context.getText())); + } + + @Override + public PathNode visitIntegerLiteral(JsonPathParser.IntegerLiteralContext context) + { + return new SqlValueLiteral(new LongLiteral(context.getText())); + } + + @Override + public PathNode visitStringLiteral(JsonPathParser.StringLiteralContext context) + { + return new SqlValueLiteral(new StringLiteral(unquote(context.STRING().getText()))); + } + + private static String unquote(String quoted) + { + return quoted.substring(1, quoted.length() - 1) + .replace("\"\"", "\""); + } + + @Override + public PathNode visitNullLiteral(JsonPathParser.NullLiteralContext context) + { + return JSON_NULL; + } + + @Override + public PathNode visitBooleanLiteral(JsonPathParser.BooleanLiteralContext context) + { + return new SqlValueLiteral(new BooleanLiteral(context.getText())); + } + + @Override + public PathNode visitContextVariable(JsonPathParser.ContextVariableContext context) + { + return new ContextVariable(); + } + + @Override + public PathNode visitNamedVariable(JsonPathParser.NamedVariableContext context) + { + return namedVariable(context.NAMED_VARIABLE()); + } + + private static NamedVariable namedVariable(TerminalNode namedVariable) + { + // drop leading `$` + return new NamedVariable(namedVariable.getText().substring(1)); + } + + @Override + public PathNode visitLastIndexVariable(JsonPathParser.LastIndexVariableContext context) + { + return new LastIndexVariable(); + } + + @Override + public PathNode visitPredicateCurrentItemVariable(JsonPathParser.PredicateCurrentItemVariableContext context) + { + return new PredicateCurrentItemVariable(); + } + + @Override + public PathNode visitParenthesizedPath(JsonPathParser.ParenthesizedPathContext context) + { + return visit(context.pathExpression()); + } + + @Override + public PathNode visitMemberAccessor(JsonPathParser.MemberAccessorContext context) + { + PathNode base = visit(context.accessorExpression()); + Optional key = Optional.empty(); + if (context.stringLiteral() != null) { + key = Optional.of(unquote(context.stringLiteral().getText())); + } + else if (context.identifier() != null) { + key = Optional.of(context.identifier().getText()); + } + return new MemberAccessor(base, key); + } + + @Override + public PathNode visitWildcardMemberAccessor(JsonPathParser.WildcardMemberAccessorContext context) + { + PathNode base = visit(context.accessorExpression()); + return new MemberAccessor(base, Optional.empty()); + } + + @Override + public PathNode visitArrayAccessor(JsonPathParser.ArrayAccessorContext context) + { + PathNode base = visit(context.accessorExpression()); + ImmutableList.Builder subscripts = ImmutableList.builder(); + for (JsonPathParser.SubscriptContext subscript : context.subscript()) { + if (subscript.singleton != null) { + subscripts.add(new ArrayAccessor.Subscript(visit(subscript.singleton))); + } + else { + subscripts.add(new ArrayAccessor.Subscript(visit(subscript.from), visit(subscript.to))); + } + } + return new ArrayAccessor(base, subscripts.build()); + } + + @Override + public PathNode visitWildcardArrayAccessor(JsonPathParser.WildcardArrayAccessorContext context) + { + PathNode base = visit(context.accessorExpression()); + return new ArrayAccessor(base, ImmutableList.of()); + } + + @Override + public PathNode visitFilter(JsonPathParser.FilterContext context) + { + PathNode base = visit(context.accessorExpression()); + Predicate predicate = (Predicate) visit(context.predicate()); + return new Filter(base, predicate); + } + + @Override + public PathNode visitTypeMethod(JsonPathParser.TypeMethodContext context) + { + PathNode base = visit(context.accessorExpression()); + return new TypeMethod(base); + } + + @Override + public PathNode visitSizeMethod(JsonPathParser.SizeMethodContext context) + { + PathNode base = visit(context.accessorExpression()); + return new SizeMethod(base); + } + + @Override + public PathNode visitDoubleMethod(JsonPathParser.DoubleMethodContext context) + { + PathNode base = visit(context.accessorExpression()); + return new DoubleMethod(base); + } + + @Override + public PathNode visitCeilingMethod(JsonPathParser.CeilingMethodContext context) + { + PathNode base = visit(context.accessorExpression()); + return new CeilingMethod(base); + } + + @Override + public PathNode visitFloorMethod(JsonPathParser.FloorMethodContext context) + { + PathNode base = visit(context.accessorExpression()); + return new FloorMethod(base); + } + + @Override + public PathNode visitAbsMethod(JsonPathParser.AbsMethodContext context) + { + PathNode base = visit(context.accessorExpression()); + return new AbsMethod(base); + } + + @Override + public PathNode visitDatetimeMethod(JsonPathParser.DatetimeMethodContext context) + { + PathNode base = visit(context.accessorExpression()); + Optional format = Optional.empty(); + if (context.stringLiteral() != null) { + format = Optional.of(unquote(context.stringLiteral().getText())); + } + return new DatetimeMethod(base, format); + } + + @Override + public PathNode visitKeyValueMethod(JsonPathParser.KeyValueMethodContext context) + { + PathNode base = visit(context.accessorExpression()); + return new KeyValueMethod(base); + } + + @Override + public PathNode visitSignedUnary(JsonPathParser.SignedUnaryContext context) + { + PathNode base = visit(context.pathExpression()); + return new ArithmeticUnary(getSign(context.sign.getText()), base); + } + + private static Sign getSign(String operator) + { + switch (operator) { + case "+": + return Sign.PLUS; + case "-": + return Sign.MINUS; + default: + throw new UnsupportedOperationException("unexpected unary operator: " + operator); + } + } + + @Override + public PathNode visitBinary(JsonPathParser.BinaryContext context) + { + PathNode left = visit(context.left); + PathNode right = visit(context.right); + return new ArithmeticBinary(getOperator(context.operator.getText()), left, right); + } + + private static Operator getOperator(String operator) + { + switch (operator) { + case "+": + return Operator.ADD; + case "-": + return Operator.SUBTRACT; + case "*": + return Operator.MULTIPLY; + case "/": + return Operator.DIVIDE; + case "%": + return Operator.MODULUS; + default: + throw new UnsupportedOperationException("unexpected binary operator: " + operator); + } + } + + // predicate + + @Override + public PathNode visitComparisonPredicate(JsonPathParser.ComparisonPredicateContext context) + { + PathNode left = visit(context.left); + PathNode right = visit(context.right); + return new ComparisonPredicate(getComparisonOperator(context.comparisonOperator().getText()), left, right); + } + + private static ComparisonPredicate.Operator getComparisonOperator(String operator) + { + switch (operator) { + case "==": + return ComparisonPredicate.Operator.EQUAL; + case "<>": + case "!=": + return ComparisonPredicate.Operator.NOT_EQUAL; + case "<": + return ComparisonPredicate.Operator.LESS_THAN; + case ">": + return ComparisonPredicate.Operator.GREATER_THAN; + case "<=": + return ComparisonPredicate.Operator.LESS_THAN_OR_EQUAL; + case ">=": + return ComparisonPredicate.Operator.GREATER_THAN_OR_EQUAL; + default: + throw new UnsupportedOperationException("unexpected comparison operator: " + operator); + } + } + + @Override + public PathNode visitConjunctionPredicate(JsonPathParser.ConjunctionPredicateContext context) + { + Predicate left = (Predicate) visit(context.left); + Predicate right = (Predicate) visit(context.right); + return new ConjunctionPredicate(left, right); + } + + @Override + public PathNode visitDisjunctionPredicate(JsonPathParser.DisjunctionPredicateContext context) + { + Predicate left = (Predicate) visit(context.left); + Predicate right = (Predicate) visit(context.right); + return new DisjunctionPredicate(left, right); + } + + @Override + public PathNode visitExistsPredicate(JsonPathParser.ExistsPredicateContext context) + { + PathNode path = visit(context.pathExpression()); + return new ExistsPredicate(path); + } + + @Override + public PathNode visitIsUnknownPredicate(JsonPathParser.IsUnknownPredicateContext context) + { + Predicate predicate = (Predicate) visit(context.predicate()); + return new IsUnknownPredicate(predicate); + } + + @Override + public PathNode visitLikeRegexPredicate(JsonPathParser.LikeRegexPredicateContext context) + { + PathNode path = visit(context.base); + String pattern = unquote(context.pattern.getText()); + Optional flag = Optional.empty(); + if (context.flag != null) { + flag = Optional.of(unquote(context.flag.getText())); + } + return new LikeRegexPredicate(path, pattern, flag); + } + + @Override + public PathNode visitNegationPredicate(JsonPathParser.NegationPredicateContext context) + { + Predicate predicate = (Predicate) visit(context.delimitedPredicate()); + return new NegationPredicate(predicate); + } + + @Override + public PathNode visitParenthesizedPredicate(JsonPathParser.ParenthesizedPredicateContext context) + { + return visit(context.predicate()); + } + + @Override + public PathNode visitStartsWithPredicate(JsonPathParser.StartsWithPredicateContext context) + { + PathNode whole = visit(context.whole); + PathNode initial; + if (context.string != null) { + initial = visit(context.string); + } + else { + initial = namedVariable(context.NAMED_VARIABLE()); + } + return new StartsWithPredicate(whole, initial); + } + + @Override + protected PathNode aggregateResult(PathNode aggregate, PathNode nextResult) + { + // do not skip over unrecognized nodes + if (nextResult == null) { + throw new UnsupportedOperationException("not yet implemented"); + } + + if (aggregate == null) { + return nextResult; + } + + throw new UnsupportedOperationException("not yet implemented"); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/AbsMethod.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/AbsMethod.java new file mode 100644 index 000000000000..7ddfb3159690 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/AbsMethod.java @@ -0,0 +1,29 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class AbsMethod + extends Method +{ + public AbsMethod(PathNode base) + { + super(base); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitAbsMethod(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Accessor.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Accessor.java new file mode 100644 index 000000000000..1c8280269b98 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Accessor.java @@ -0,0 +1,38 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public abstract class Accessor + extends PathNode +{ + protected final PathNode base; + + Accessor(PathNode base) + { + this.base = requireNonNull(base, "accessor base is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitAccessor(this, context); + } + + public PathNode getBase() + { + return base; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArithmeticBinary.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArithmeticBinary.java new file mode 100644 index 000000000000..4010f0315069 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArithmeticBinary.java @@ -0,0 +1,61 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class ArithmeticBinary + extends PathNode +{ + private final Operator operator; + private final PathNode left; + private final PathNode right; + + public ArithmeticBinary(Operator operator, PathNode left, PathNode right) + { + this.operator = requireNonNull(operator, "operator is null"); + this.left = requireNonNull(left, "left is null"); + this.right = requireNonNull(right, "right is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitArithmeticBinary(this, context); + } + + public Operator getOperator() + { + return operator; + } + + public PathNode getLeft() + { + return left; + } + + public PathNode getRight() + { + return right; + } + + public enum Operator + { + ADD, + SUBTRACT, + MULTIPLY, + DIVIDE, + MODULUS + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArithmeticUnary.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArithmeticUnary.java new file mode 100644 index 000000000000..1bbf0be704a2 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArithmeticUnary.java @@ -0,0 +1,51 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class ArithmeticUnary + extends PathNode +{ + private final Sign sign; + private final PathNode base; + + public ArithmeticUnary(Sign sign, PathNode base) + { + this.sign = requireNonNull(sign, "sign is null"); + this.base = requireNonNull(base, "base is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitArithmeticUnary(this, context); + } + + public Sign getSign() + { + return sign; + } + + public PathNode getBase() + { + return base; + } + + public enum Sign + { + PLUS, + MINUS + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArrayAccessor.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArrayAccessor.java new file mode 100644 index 000000000000..1e9b7b84b958 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ArrayAccessor.java @@ -0,0 +1,70 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import java.util.List; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class ArrayAccessor + extends Accessor +{ + private final List subscripts; + + public ArrayAccessor(PathNode base, List subscripts) + { + super(base); + this.subscripts = requireNonNull(subscripts, "subscripts is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitArrayAccessor(this, context); + } + + public List getSubscripts() + { + return subscripts; + } + + public static class Subscript + { + private final PathNode from; + private final Optional to; + + public Subscript(PathNode from) + { + this.from = requireNonNull(from, "from is null"); + this.to = Optional.empty(); + } + + public Subscript(PathNode from, PathNode to) + { + this.from = requireNonNull(from, "from is null"); + this.to = Optional.of(requireNonNull(to, "to is null")); + } + + public PathNode getFrom() + { + return from; + } + + public Optional getTo() + { + return to; + } + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/CeilingMethod.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/CeilingMethod.java new file mode 100644 index 000000000000..003961a8f158 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/CeilingMethod.java @@ -0,0 +1,29 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class CeilingMethod + extends Method +{ + public CeilingMethod(PathNode base) + { + super(base); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitCeilingMethod(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ComparisonPredicate.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ComparisonPredicate.java new file mode 100644 index 000000000000..c8f6a5af3cc4 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ComparisonPredicate.java @@ -0,0 +1,62 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class ComparisonPredicate + extends Predicate +{ + private final Operator operator; + private final PathNode left; + private final PathNode right; + + public ComparisonPredicate(Operator operator, PathNode left, PathNode right) + { + this.operator = requireNonNull(operator, "operator is null"); + this.left = requireNonNull(left, "left is null"); + this.right = requireNonNull(right, "right is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitComparisonPredicate(this, context); + } + + public Operator getOperator() + { + return operator; + } + + public PathNode getLeft() + { + return left; + } + + public PathNode getRight() + { + return right; + } + + public enum Operator + { + EQUAL, + NOT_EQUAL, + LESS_THAN, + GREATER_THAN, + LESS_THAN_OR_EQUAL, + GREATER_THAN_OR_EQUAL; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ConjunctionPredicate.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ConjunctionPredicate.java new file mode 100644 index 000000000000..35ba1b2ba2cc --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ConjunctionPredicate.java @@ -0,0 +1,45 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class ConjunctionPredicate + extends Predicate +{ + private final Predicate left; + private final Predicate right; + + public ConjunctionPredicate(Predicate left, Predicate right) + { + this.left = requireNonNull(left, "left is null"); + this.right = requireNonNull(right, "right is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitConjunctionPredicate(this, context); + } + + public Predicate getLeft() + { + return left; + } + + public Predicate getRight() + { + return right; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ContextVariable.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ContextVariable.java new file mode 100644 index 000000000000..aef058e7c44a --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ContextVariable.java @@ -0,0 +1,24 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class ContextVariable + extends PathNode +{ + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitContextVariable(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DatetimeMethod.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DatetimeMethod.java new file mode 100644 index 000000000000..becec84756f8 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DatetimeMethod.java @@ -0,0 +1,41 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class DatetimeMethod + extends Method +{ + private final Optional format; + + public DatetimeMethod(PathNode base, Optional format) + { + super(base); + this.format = requireNonNull(format, "format is null"); // TODO in IR, translate to input for java.time and create a formatter. + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitDatetimeMethod(this, context); + } + + public Optional getFormat() + { + return format; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DisjunctionPredicate.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DisjunctionPredicate.java new file mode 100644 index 000000000000..859026697750 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DisjunctionPredicate.java @@ -0,0 +1,45 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class DisjunctionPredicate + extends Predicate +{ + private final Predicate left; + private final Predicate right; + + public DisjunctionPredicate(Predicate left, Predicate right) + { + this.left = requireNonNull(left, "left is null"); + this.right = requireNonNull(right, "right is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitDisjunctionPredicate(this, context); + } + + public Predicate getLeft() + { + return left; + } + + public Predicate getRight() + { + return right; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DoubleMethod.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DoubleMethod.java new file mode 100644 index 000000000000..f15c0fda5b2c --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/DoubleMethod.java @@ -0,0 +1,29 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class DoubleMethod + extends Method +{ + public DoubleMethod(PathNode base) + { + super(base); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitDoubleMethod(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ExistsPredicate.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ExistsPredicate.java new file mode 100644 index 000000000000..51788fbdcf8a --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/ExistsPredicate.java @@ -0,0 +1,38 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class ExistsPredicate + extends Predicate +{ + private final PathNode path; + + public ExistsPredicate(PathNode path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitExistsPredicate(this, context); + } + + public PathNode getPath() + { + return path; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Filter.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Filter.java new file mode 100644 index 000000000000..7c1bd9ce4b29 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Filter.java @@ -0,0 +1,39 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class Filter + extends Accessor +{ + private final Predicate predicate; + + public Filter(PathNode base, Predicate predicate) + { + super(base); + this.predicate = requireNonNull(predicate, "predicate is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitFilter(this, context); + } + + public Predicate getPredicate() + { + return predicate; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/FloorMethod.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/FloorMethod.java new file mode 100644 index 000000000000..bdc62e97c5e4 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/FloorMethod.java @@ -0,0 +1,29 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class FloorMethod + extends Method +{ + public FloorMethod(PathNode base) + { + super(base); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitFloorMethod(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/IsUnknownPredicate.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/IsUnknownPredicate.java new file mode 100644 index 000000000000..6a26987e3287 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/IsUnknownPredicate.java @@ -0,0 +1,38 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class IsUnknownPredicate + extends Predicate +{ + private final Predicate predicate; + + public IsUnknownPredicate(Predicate predicate) + { + this.predicate = requireNonNull(predicate, "predicate is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitIsUnknownPredicate(this, context); + } + + public Predicate getPredicate() + { + return predicate; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonNullLiteral.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonNullLiteral.java new file mode 100644 index 000000000000..32e847bd3e2b --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonNullLiteral.java @@ -0,0 +1,28 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class JsonNullLiteral + extends Literal +{ + public static final JsonNullLiteral JSON_NULL = new JsonNullLiteral(); + + private JsonNullLiteral() {} + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitJsonNullLiteral(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonPath.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonPath.java new file mode 100644 index 000000000000..5b4ebb03e3a6 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonPath.java @@ -0,0 +1,45 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class JsonPath + extends PathNode +{ + private final boolean lax; + private final PathNode root; + + public JsonPath(boolean lax, PathNode root) + { + this.lax = lax; + this.root = requireNonNull(root, "root is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitJsonPath(this, context); + } + + public boolean isLax() + { + return lax; + } + + public PathNode getRoot() + { + return root; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonPathTreeVisitor.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonPathTreeVisitor.java new file mode 100644 index 000000000000..beabf01e56f1 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/JsonPathTreeVisitor.java @@ -0,0 +1,194 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import javax.annotation.Nullable; + +public abstract class JsonPathTreeVisitor +{ + public R process(PathNode node) + { + return process(node, null); + } + + public R process(PathNode node, @Nullable C context) + { + return node.accept(this, context); + } + + protected R visitPathNode(PathNode node, C context) + { + return null; + } + + protected R visitAbsMethod(AbsMethod node, C context) + { + return visitMethod(node, context); + } + + protected R visitAccessor(Accessor node, C context) + { + return visitPathNode(node, context); + } + + protected R visitArithmeticBinary(ArithmeticBinary node, C context) + { + return visitPathNode(node, context); + } + + protected R visitArithmeticUnary(ArithmeticUnary node, C context) + { + return visitPathNode(node, context); + } + + protected R visitArrayAccessor(ArrayAccessor node, C context) + { + return visitAccessor(node, context); + } + + protected R visitCeilingMethod(CeilingMethod node, C context) + { + return visitMethod(node, context); + } + + protected R visitComparisonPredicate(ComparisonPredicate node, C context) + { + return visitPredicate(node, context); + } + + protected R visitConjunctionPredicate(ConjunctionPredicate node, C context) + { + return visitPredicate(node, context); + } + + protected R visitContextVariable(ContextVariable node, C context) + { + return visitPathNode(node, context); + } + + protected R visitDatetimeMethod(DatetimeMethod node, C context) + { + return visitMethod(node, context); + } + + protected R visitDisjunctionPredicate(DisjunctionPredicate node, C context) + { + return visitPredicate(node, context); + } + + protected R visitDoubleMethod(DoubleMethod node, C context) + { + return visitMethod(node, context); + } + + protected R visitExistsPredicate(ExistsPredicate node, C context) + { + return visitPredicate(node, context); + } + + protected R visitFilter(Filter node, C context) + { + return visitAccessor(node, context); + } + + protected R visitFloorMethod(FloorMethod node, C context) + { + return visitMethod(node, context); + } + + protected R visitIsUnknownPredicate(IsUnknownPredicate node, C context) + { + return visitPredicate(node, context); + } + + protected R visitJsonNullLiteral(JsonNullLiteral node, C context) + { + return visitLiteral(node, context); + } + + protected R visitJsonPath(JsonPath node, C context) + { + return visitPathNode(node, context); + } + + protected R visitKeyValueMethod(KeyValueMethod node, C context) + { + return visitMethod(node, context); + } + + protected R visitLastIndexVariable(LastIndexVariable node, C context) + { + return visitPathNode(node, context); + } + + protected R visitLikeRegexPredicate(LikeRegexPredicate node, C context) + { + return visitPredicate(node, context); + } + + protected R visitLiteral(Literal node, C context) + { + return visitPathNode(node, context); + } + + protected R visitMemberAccessor(MemberAccessor node, C context) + { + return visitAccessor(node, context); + } + + protected R visitMethod(Method node, C context) + { + return visitAccessor(node, context); + } + + protected R visitNamedVariable(NamedVariable node, C context) + { + return visitPathNode(node, context); + } + + protected R visitNegationPredicate(NegationPredicate node, C context) + { + return visitPredicate(node, context); + } + + protected R visitPredicate(Predicate node, C context) + { + return visitPathNode(node, context); + } + + protected R visitPredicateCurrentItemVariable(PredicateCurrentItemVariable node, C context) + { + return visitPathNode(node, context); + } + + protected R visitSizeMethod(SizeMethod node, C context) + { + return visitMethod(node, context); + } + + protected R visitSqlValueLiteral(SqlValueLiteral node, C context) + { + return visitLiteral(node, context); + } + + protected R visitStartsWithPredicate(StartsWithPredicate node, C context) + { + return visitPredicate(node, context); + } + + protected R visitTypeMethod(TypeMethod node, C context) + { + return visitMethod(node, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/KeyValueMethod.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/KeyValueMethod.java new file mode 100644 index 000000000000..a475926ede69 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/KeyValueMethod.java @@ -0,0 +1,29 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class KeyValueMethod + extends Method +{ + public KeyValueMethod(PathNode base) + { + super(base); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitKeyValueMethod(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/LastIndexVariable.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/LastIndexVariable.java new file mode 100644 index 000000000000..a34dbaf981e8 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/LastIndexVariable.java @@ -0,0 +1,24 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class LastIndexVariable + extends PathNode +{ + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitLastIndexVariable(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/LikeRegexPredicate.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/LikeRegexPredicate.java new file mode 100644 index 000000000000..5bbaf117f114 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/LikeRegexPredicate.java @@ -0,0 +1,54 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class LikeRegexPredicate + extends Predicate +{ + private final PathNode path; + private final String pattern; + private final Optional flag; + + public LikeRegexPredicate(PathNode path, String pattern, Optional flag) + { + this.path = requireNonNull(path, "path is null"); + this.pattern = requireNonNull(pattern, "pattern is null"); + this.flag = requireNonNull(flag, "flag is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitLikeRegexPredicate(this, context); + } + + public PathNode getPath() + { + return path; + } + + public String getPattern() + { + return pattern; + } + + public Optional getFlag() + { + return flag; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Literal.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Literal.java new file mode 100644 index 000000000000..fc76d3b68bce --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Literal.java @@ -0,0 +1,24 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public abstract class Literal + extends PathNode +{ + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitLiteral(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/MemberAccessor.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/MemberAccessor.java new file mode 100644 index 000000000000..0e3ea555de9a --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/MemberAccessor.java @@ -0,0 +1,41 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class MemberAccessor + extends Accessor +{ + private final Optional key; + + public MemberAccessor(PathNode base, Optional key) + { + super(base); + this.key = requireNonNull(key, "key is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitMemberAccessor(this, context); + } + + public Optional getKey() + { + return key; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Method.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Method.java new file mode 100644 index 000000000000..a3199e157559 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Method.java @@ -0,0 +1,29 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public abstract class Method + extends Accessor +{ + Method(PathNode base) + { + super(base); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitMethod(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/NamedVariable.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/NamedVariable.java new file mode 100644 index 000000000000..99a86c9bd5de --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/NamedVariable.java @@ -0,0 +1,38 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class NamedVariable + extends PathNode +{ + private final String name; + + public NamedVariable(String name) + { + this.name = requireNonNull(name, "name is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitNamedVariable(this, context); + } + + public String getName() + { + return name; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/NegationPredicate.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/NegationPredicate.java new file mode 100644 index 000000000000..2df191fa6ad9 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/NegationPredicate.java @@ -0,0 +1,38 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class NegationPredicate + extends Predicate +{ + private final Predicate predicate; + + public NegationPredicate(Predicate predicate) + { + this.predicate = requireNonNull(predicate, "predicate is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitNegationPredicate(this, context); + } + + public Predicate getPredicate() + { + return predicate; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/PathNode.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/PathNode.java new file mode 100644 index 000000000000..6ec60ef0b0c2 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/PathNode.java @@ -0,0 +1,22 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public abstract class PathNode +{ + protected R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitPathNode(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Predicate.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Predicate.java new file mode 100644 index 000000000000..a35c5e80ecd9 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/Predicate.java @@ -0,0 +1,24 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public abstract class Predicate + extends PathNode +{ + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitPredicate(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/PredicateCurrentItemVariable.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/PredicateCurrentItemVariable.java new file mode 100644 index 000000000000..9b617f889d1a --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/PredicateCurrentItemVariable.java @@ -0,0 +1,24 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class PredicateCurrentItemVariable + extends PathNode +{ + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitPredicateCurrentItemVariable(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/SizeMethod.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/SizeMethod.java new file mode 100644 index 000000000000..b0315f24f193 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/SizeMethod.java @@ -0,0 +1,29 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class SizeMethod + extends Method +{ + public SizeMethod(PathNode base) + { + super(base); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitSizeMethod(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/SqlValueLiteral.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/SqlValueLiteral.java new file mode 100644 index 000000000000..d79feb5195f6 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/SqlValueLiteral.java @@ -0,0 +1,39 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import static java.util.Objects.requireNonNull; + +public class SqlValueLiteral + extends Literal +{ + private final io.trino.sql.tree.Literal value; + + public SqlValueLiteral(io.trino.sql.tree.Literal value) + { + super(); + this.value = requireNonNull(value, "value is null"); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitSqlValueLiteral(this, context); + } + + public io.trino.sql.tree.Literal getValue() + { + return value; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/StartsWithPredicate.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/StartsWithPredicate.java new file mode 100644 index 000000000000..54c46057eaf6 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/StartsWithPredicate.java @@ -0,0 +1,52 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +import io.trino.sql.tree.StringLiteral; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class StartsWithPredicate + extends Predicate +{ + private final PathNode whole; + private final PathNode initial; + + public StartsWithPredicate(PathNode whole, PathNode initial) + { + requireNonNull(whole, "whole is null"); + requireNonNull(initial, "initial is null"); + checkArgument(initial instanceof NamedVariable || (initial instanceof SqlValueLiteral && ((SqlValueLiteral) initial).getValue() instanceof StringLiteral), "initial must be a named variable or a string literal"); + + this.whole = whole; + this.initial = initial; + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitStartsWithPredicate(this, context); + } + + public PathNode getWhole() + { + return whole; + } + + public PathNode getInitial() + { + return initial; + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/TypeMethod.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/TypeMethod.java new file mode 100644 index 000000000000..0c427a8a6a1b --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/tree/TypeMethod.java @@ -0,0 +1,29 @@ +/* + * 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 io.trino.sql.jsonpath.tree; + +public class TypeMethod + extends Method +{ + public TypeMethod(PathNode base) + { + super(base); + } + + @Override + public R accept(JsonPathTreeVisitor visitor, C context) + { + return visitor.visitTypeMethod(this, context); + } +} diff --git a/core/trino-parser/src/main/java/io/trino/sql/parser/ParsingException.java b/core/trino-parser/src/main/java/io/trino/sql/parser/ParsingException.java index 3882d54f09ce..443123ec4882 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/parser/ParsingException.java +++ b/core/trino-parser/src/main/java/io/trino/sql/parser/ParsingException.java @@ -40,6 +40,11 @@ public ParsingException(String message) this(message, null, 1, 1); } + public ParsingException(String message, RecognitionException cause) + { + this(message, cause, 1, 1); + } + public ParsingException(String message, NodeLocation nodeLocation) { this(message, null, nodeLocation.getLineNumber(), nodeLocation.getColumnNumber()); diff --git a/core/trino-parser/src/main/java/io/trino/sql/tree/DefaultTraversalVisitor.java b/core/trino-parser/src/main/java/io/trino/sql/tree/DefaultTraversalVisitor.java index a4c89adff8ee..034d6054c428 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/tree/DefaultTraversalVisitor.java +++ b/core/trino-parser/src/main/java/io/trino/sql/tree/DefaultTraversalVisitor.java @@ -920,4 +920,41 @@ protected Void visitLabelDereference(LabelDereference node, C context) return null; } + + @Override + protected Void visitJsonExists(JsonExists node, C context) + { + process(node.getJsonPathInvocation(), context); + + return null; + } + + @Override + protected Void visitJsonValue(JsonValue node, C context) + { + process(node.getJsonPathInvocation(), context); + node.getEmptyDefault().ifPresent(expression -> process(expression, context)); + node.getErrorDefault().ifPresent(expression -> process(expression, context)); + + return null; + } + + @Override + protected Void visitJsonQuery(JsonQuery node, C context) + { + process(node.getJsonPathInvocation(), context); + + return null; + } + + @Override + protected Void visitJsonPathInvocation(JsonPathInvocation node, C context) + { + process(node.getInputExpression(), context); + for (JsonPathParameter parameter : node.getPathParameters()) { + process(parameter.getParameter(), context); + } + + return null; + } } diff --git a/core/trino-parser/src/main/java/io/trino/sql/tree/ExpressionRewriter.java b/core/trino-parser/src/main/java/io/trino/sql/tree/ExpressionRewriter.java index 6864375b2b0a..39c692d6113d 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/tree/ExpressionRewriter.java +++ b/core/trino-parser/src/main/java/io/trino/sql/tree/ExpressionRewriter.java @@ -269,4 +269,19 @@ public Expression rewriteLabelDereference(LabelDereference node, C context, Expr { return rewriteExpression(node, context, treeRewriter); } + + public Expression rewriteJsonExists(JsonExists node, C context, ExpressionTreeRewriter treeRewriter) + { + return rewriteExpression(node, context, treeRewriter); + } + + public Expression rewriteJsonValue(JsonValue node, C context, ExpressionTreeRewriter treeRewriter) + { + return rewriteExpression(node, context, treeRewriter); + } + + public Expression rewriteJsonQuery(JsonQuery node, C context, ExpressionTreeRewriter treeRewriter) + { + return rewriteExpression(node, context, treeRewriter); + } } diff --git a/core/trino-parser/src/main/java/io/trino/sql/tree/ExpressionTreeRewriter.java b/core/trino-parser/src/main/java/io/trino/sql/tree/ExpressionTreeRewriter.java index b5bd2051042a..846aac09ca11 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/tree/ExpressionTreeRewriter.java +++ b/core/trino-parser/src/main/java/io/trino/sql/tree/ExpressionTreeRewriter.java @@ -1161,6 +1161,104 @@ protected Expression visitLabelDereference(LabelDereference node, Context con return node; } + + @Override + protected Expression visitJsonExists(JsonExists node, Context context) + { + if (!context.isDefaultRewrite()) { + Expression result = rewriter.rewriteJsonExists(node, context.get(), ExpressionTreeRewriter.this); + if (result != null) { + return result; + } + } + + JsonPathInvocation jsonPathInvocation = rewriteJsonPathInvocation(node.getJsonPathInvocation(), context); + + if (node.getJsonPathInvocation() != jsonPathInvocation) { + return new JsonExists(node.getLocation(), jsonPathInvocation, node.getErrorBehavior()); + } + + return node; + } + + @Override + protected Expression visitJsonValue(JsonValue node, Context context) + { + if (!context.isDefaultRewrite()) { + Expression result = rewriter.rewriteJsonValue(node, context.get(), ExpressionTreeRewriter.this); + if (result != null) { + return result; + } + } + + JsonPathInvocation jsonPathInvocation = rewriteJsonPathInvocation(node.getJsonPathInvocation(), context); + + Optional emptyDefault = node.getEmptyDefault().map(expression -> rewrite(expression, context.get())); + Optional errorDefault = node.getErrorDefault().map(expression -> rewrite(expression, context.get())); + + if (node.getJsonPathInvocation() != jsonPathInvocation || + !sameElements(node.getEmptyDefault(), emptyDefault) || + !sameElements(node.getErrorDefault(), errorDefault)) { + return new JsonValue( + node.getLocation(), + jsonPathInvocation, + node.getReturnedType(), + node.getEmptyBehavior(), + emptyDefault, + node.getErrorBehavior(), + errorDefault); + } + + return node; + } + + @Override + protected Expression visitJsonQuery(JsonQuery node, Context context) + { + if (!context.isDefaultRewrite()) { + Expression result = rewriter.rewriteJsonQuery(node, context.get(), ExpressionTreeRewriter.this); + if (result != null) { + return result; + } + } + + JsonPathInvocation jsonPathInvocation = rewriteJsonPathInvocation(node.getJsonPathInvocation(), context); + + if (node.getJsonPathInvocation() != jsonPathInvocation) { + return new JsonQuery( + node.getLocation(), + jsonPathInvocation, + node.getReturnedType(), + node.getOutputFormat(), + node.getWrapperBehavior(), + node.getQuotesBehavior(), + node.getEmptyBehavior(), + node.getErrorBehavior()); + } + + return node; + } + + private JsonPathInvocation rewriteJsonPathInvocation(JsonPathInvocation pathInvocation, Context context) + { + Expression inputExpression = rewrite(pathInvocation.getInputExpression(), context.get()); + + List pathParameters = pathInvocation.getPathParameters().stream() + .map(pathParameter -> { + Expression expression = rewrite(pathParameter.getParameter(), context.get()); + if (pathParameter.getParameter() != expression) { + return new JsonPathParameter(pathParameter.getLocation(), pathParameter.getName(), expression, pathParameter.getFormat()); + } + return pathParameter; + }) + .collect(toImmutableList()); + + if (pathInvocation.getInputExpression() != inputExpression || !sameElements(pathInvocation.getPathParameters(), pathParameters)) { + return new JsonPathInvocation(pathInvocation.getLocation(), inputExpression, pathInvocation.getInputFormat(), pathInvocation.getJsonPath(), pathParameters); + } + + return pathInvocation; + } } public static class Context diff --git a/core/trino-spi/src/main/java/io/trino/spi/StandardErrorCode.java b/core/trino-spi/src/main/java/io/trino/spi/StandardErrorCode.java index 845b96f9e346..dfe82029258e 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/StandardErrorCode.java +++ b/core/trino-spi/src/main/java/io/trino/spi/StandardErrorCode.java @@ -133,6 +133,10 @@ public enum StandardErrorCode MISSING_RETURN_TYPE(109, USER_ERROR), AMBIGUOUS_RETURN_TYPE(110, USER_ERROR), MISSING_ARGUMENT(111, USER_ERROR), + DUPLICATE_PARAMETER_NAME(112, USER_ERROR), + INVALID_PATH(113, USER_ERROR), + JSON_INPUT_CONVERSION_ERROR(114, USER_ERROR), + JSON_OUTPUT_CONVERSION_ERROR(115, USER_ERROR), GENERIC_INTERNAL_ERROR(65536, INTERNAL_ERROR), TOO_MANY_REQUESTS_FAILED(65537, INTERNAL_ERROR), diff --git a/core/trino-spi/src/main/java/io/trino/spi/type/StandardTypes.java b/core/trino-spi/src/main/java/io/trino/spi/type/StandardTypes.java index eba59076cbd5..e39e7ba6bde9 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/type/StandardTypes.java +++ b/core/trino-spi/src/main/java/io/trino/spi/type/StandardTypes.java @@ -41,6 +41,7 @@ public final class StandardTypes public static final String ARRAY = "array"; public static final String MAP = "map"; public static final String JSON = "json"; + public static final String JSON_2016 = "json2016"; public static final String IPADDRESS = "ipaddress"; public static final String GEOMETRY = "Geometry"; public static final String UUID = "uuid";