Skip to content

Commit

Permalink
Analyze JSON_EXISTS, JSON_VALUE and JSON_QUERY functions
Browse files Browse the repository at this point in the history
  • Loading branch information
kasiafi committed May 27, 2022
1 parent 5a0dcb9 commit 73c4617
Show file tree
Hide file tree
Showing 64 changed files with 4,911 additions and 3 deletions.
169 changes: 169 additions & 0 deletions core/trino-main/src/main/java/io/trino/json/JsonInputErrorNode.java
Original file line number Diff line number Diff line change
@@ -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 extends JsonNode> 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<JsonNode> findValues(String fieldName, List<JsonNode> foundSoFar)
{
throw new UnsupportedOperationException();
}

@Override
public List<String> findValuesAsText(String fieldName, List<String> foundSoFar)
{
throw new UnsupportedOperationException();
}

@Override
public List<JsonNode> findParents(String fieldName, List<JsonNode> 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();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
@@ -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");
}
}
Original file line number Diff line number Diff line change
@@ -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);
}
}
Loading

0 comments on commit 73c4617

Please sign in to comment.