Skip to content

Commit

Permalink
Do not share exceptions on static fields
Browse files Browse the repository at this point in the history
Exception instances are mutable.
  • Loading branch information
findepi committed May 15, 2023
1 parent 120ec44 commit 1389246
Show file tree
Hide file tree
Showing 5 changed files with 43 additions and 46 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import java.lang.invoke.MethodHandle;
import java.util.List;
import java.util.Optional;
import java.util.function.Supplier;

import static io.trino.json.JsonInputErrorNode.JSON_ERROR;
import static io.trino.operator.scalar.json.ParameterUtil.getParametersArray;
Expand All @@ -58,8 +59,6 @@ public class JsonExistsFunction
{
public static final String JSON_EXISTS_FUNCTION_NAME = "$json_exists";
private static final MethodHandle METHOD_HANDLE = methodHandle(JsonExistsFunction.class, "jsonExists", FunctionManager.class, Metadata.class, TypeManager.class, Type.class, JsonPathInvocationContext.class, ConnectorSession.class, JsonNode.class, IrJsonPath.class, Block.class, long.class);
private static final TrinoException INPUT_ARGUMENT_ERROR = new JsonInputConversionError("malformed input argument to JSON_EXISTS function");
private static final TrinoException PATH_PARAMETER_ERROR = new JsonInputConversionError("malformed JSON path parameter to JSON_EXISTS function");

private final FunctionManager functionManager;
private final Metadata metadata;
Expand Down Expand Up @@ -117,12 +116,12 @@ public static Boolean jsonExists(
long errorBehavior)
{
if (inputExpression.equals(JSON_ERROR)) {
return handleError(errorBehavior, INPUT_ARGUMENT_ERROR); // ERROR ON ERROR was already handled by the input function
return handleError(errorBehavior, () -> new JsonInputConversionError("malformed input argument to JSON_EXISTS function")); // ERROR ON ERROR was already handled by the input function
}
Object[] parameters = getParametersArray(parametersRowType, parametersRow);
for (Object parameter : parameters) {
if (parameter.equals(JSON_ERROR)) {
return handleError(errorBehavior, PATH_PARAMETER_ERROR); // ERROR ON ERROR was already handled by the input function
return handleError(errorBehavior, () -> new JsonInputConversionError("malformed JSON path parameter to JSON_EXISTS function")); // ERROR ON ERROR was already handled by the input function
}
}
// The jsonPath argument is constant for every row. We use the first incoming jsonPath argument to initialize
Expand All @@ -138,13 +137,13 @@ public static Boolean jsonExists(
pathResult = evaluator.evaluate(inputExpression, parameters);
}
catch (PathEvaluationError e) {
return handleError(errorBehavior, e);
return handleError(errorBehavior, () -> e);
}

return !pathResult.isEmpty();
}

private static Boolean handleError(long errorBehavior, TrinoException error)
private static Boolean handleError(long errorBehavior, Supplier<TrinoException> error)
{
switch (ErrorBehavior.values()[(int) errorBehavior]) {
case FALSE:
Expand All @@ -154,7 +153,7 @@ private static Boolean handleError(long errorBehavior, TrinoException error)
case UNKNOWN:
return null;
case ERROR:
throw error;
throw error.get();
}
throw new IllegalStateException("unexpected error behavior");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import java.lang.invoke.MethodHandle;
import java.util.List;
import java.util.Optional;
import java.util.function.Supplier;

import static io.trino.json.JsonInputErrorNode.JSON_ERROR;
import static io.trino.json.ir.SqlJsonLiteralConverter.getJsonNode;
Expand All @@ -66,10 +67,6 @@ public class JsonQueryFunction
private static final MethodHandle METHOD_HANDLE = methodHandle(JsonQueryFunction.class, "jsonQuery", FunctionManager.class, Metadata.class, TypeManager.class, Type.class, JsonPathInvocationContext.class, ConnectorSession.class, JsonNode.class, IrJsonPath.class, Block.class, long.class, long.class, long.class);
private static final JsonNode EMPTY_ARRAY_RESULT = new ArrayNode(JsonNodeFactory.instance);
private static final JsonNode EMPTY_OBJECT_RESULT = new ObjectNode(JsonNodeFactory.instance);
private static final TrinoException INPUT_ARGUMENT_ERROR = new JsonInputConversionError("malformed input argument to JSON_QUERY function");
private static final TrinoException PATH_PARAMETER_ERROR = new JsonInputConversionError("malformed JSON path parameter to JSON_QUERY function");
private static final TrinoException NO_ITEMS = new JsonOutputConversionError("JSON path found no items");
private static final TrinoException MULTIPLE_ITEMS = new JsonOutputConversionError("JSON path found multiple items");

private final FunctionManager functionManager;
private final Metadata metadata;
Expand Down Expand Up @@ -135,12 +132,12 @@ public static JsonNode jsonQuery(
long errorBehavior)
{
if (inputExpression.equals(JSON_ERROR)) {
return handleSpecialCase(errorBehavior, INPUT_ARGUMENT_ERROR); // ERROR ON ERROR was already handled by the input function
return handleSpecialCase(errorBehavior, () -> new JsonInputConversionError("malformed input argument to JSON_QUERY function")); // ERROR ON ERROR was already handled by the input function
}
Object[] parameters = getParametersArray(parametersRowType, parametersRow);
for (Object parameter : parameters) {
if (parameter.equals(JSON_ERROR)) {
return handleSpecialCase(errorBehavior, PATH_PARAMETER_ERROR); // ERROR ON ERROR was already handled by the input function
return handleSpecialCase(errorBehavior, () -> new JsonInputConversionError("malformed JSON path parameter to JSON_QUERY function")); // ERROR ON ERROR was already handled by the input function
}
}
// The jsonPath argument is constant for every row. We use the first incoming jsonPath argument to initialize
Expand All @@ -156,12 +153,12 @@ public static JsonNode jsonQuery(
pathResult = evaluator.evaluate(inputExpression, parameters);
}
catch (PathEvaluationError e) {
return handleSpecialCase(errorBehavior, e);
return handleSpecialCase(errorBehavior, () -> e);
}

// handle empty sequence
if (pathResult.isEmpty()) {
return handleSpecialCase(emptyBehavior, NO_ITEMS);
return handleSpecialCase(emptyBehavior, () -> new JsonOutputConversionError("JSON path found no items"));
}

// translate sequence to JSON items
Expand All @@ -170,7 +167,7 @@ public static JsonNode jsonQuery(
if (item instanceof TypedValue) {
Optional<JsonNode> jsonNode = getJsonNode((TypedValue) item);
if (jsonNode.isEmpty()) {
return handleSpecialCase(errorBehavior, new JsonOutputConversionError(format(
return handleSpecialCase(errorBehavior, () -> new JsonOutputConversionError(format(
"JSON path returned a scalar SQL value of type %s that cannot be represented as JSON",
((TypedValue) item).getType())));
}
Expand Down Expand Up @@ -205,16 +202,16 @@ public static JsonNode jsonQuery(
// if the only item is a TextNode, need to apply the KEEP / OMIT QUOTES behavior. this is done by the JSON output function
}

return handleSpecialCase(errorBehavior, MULTIPLE_ITEMS);
return handleSpecialCase(errorBehavior, () -> new JsonOutputConversionError("JSON path found multiple items"));
}

private static JsonNode handleSpecialCase(long behavior, TrinoException error)
private static JsonNode handleSpecialCase(long behavior, Supplier<TrinoException> error)
{
switch (EmptyOrErrorBehavior.values()[(int) behavior]) {
case NULL:
return null;
case ERROR:
throw error;
throw error.get();
case EMPTY_ARRAY:
return EMPTY_ARRAY_RESULT;
case EMPTY_OBJECT:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import java.lang.invoke.MethodHandle;
import java.util.List;
import java.util.Optional;
import java.util.function.Supplier;

import static com.google.common.collect.Iterables.getOnlyElement;
import static io.trino.json.JsonInputErrorNode.JSON_ERROR;
Expand All @@ -73,11 +74,6 @@ public class JsonValueFunction
private static final MethodHandle METHOD_HANDLE_BOOLEAN = methodHandle(JsonValueFunction.class, "jsonValueBoolean", FunctionManager.class, Metadata.class, TypeManager.class, Type.class, Type.class, JsonPathInvocationContext.class, ConnectorSession.class, JsonNode.class, IrJsonPath.class, Block.class, long.class, Boolean.class, long.class, Boolean.class);
private static final MethodHandle METHOD_HANDLE_SLICE = methodHandle(JsonValueFunction.class, "jsonValueSlice", FunctionManager.class, Metadata.class, TypeManager.class, Type.class, Type.class, JsonPathInvocationContext.class, ConnectorSession.class, JsonNode.class, IrJsonPath.class, Block.class, long.class, Slice.class, long.class, Slice.class);
private static final MethodHandle METHOD_HANDLE = methodHandle(JsonValueFunction.class, "jsonValue", FunctionManager.class, Metadata.class, TypeManager.class, Type.class, Type.class, JsonPathInvocationContext.class, ConnectorSession.class, JsonNode.class, IrJsonPath.class, Block.class, long.class, Object.class, long.class, Object.class);
private static final TrinoException INPUT_ARGUMENT_ERROR = new JsonInputConversionError("malformed input argument to JSON_VALUE function");
private static final TrinoException PATH_PARAMETER_ERROR = new JsonInputConversionError("malformed JSON path parameter to JSON_VALUE function");
private static final TrinoException NO_ITEMS = new JsonValueResultError("JSON path found no items");
private static final TrinoException MULTIPLE_ITEMS = new JsonValueResultError("JSON path found multiple items");
private static final TrinoException INCONVERTIBLE_ITEM = new JsonValueResultError("JSON path found an item that cannot be converted to an SQL value");

private final FunctionManager functionManager;
private final Metadata metadata;
Expand Down Expand Up @@ -246,12 +242,12 @@ public static Object jsonValue(
Object errorDefault)
{
if (inputExpression.equals(JSON_ERROR)) {
return handleSpecialCase(errorBehavior, errorDefault, INPUT_ARGUMENT_ERROR); // ERROR ON ERROR was already handled by the input function
return handleSpecialCase(errorBehavior, errorDefault, () -> new JsonInputConversionError("malformed input argument to JSON_VALUE function")); // ERROR ON ERROR was already handled by the input function
}
Object[] parameters = getParametersArray(parametersRowType, parametersRow);
for (Object parameter : parameters) {
if (parameter.equals(JSON_ERROR)) {
return handleSpecialCase(errorBehavior, errorDefault, PATH_PARAMETER_ERROR); // ERROR ON ERROR was already handled by the input function
return handleSpecialCase(errorBehavior, errorDefault, () -> new JsonInputConversionError("malformed JSON path parameter to JSON_VALUE function")); // ERROR ON ERROR was already handled by the input function
}
}
// The jsonPath argument is constant for every row. We use the first incoming jsonPath argument to initialize
Expand All @@ -267,15 +263,15 @@ public static Object jsonValue(
pathResult = evaluator.evaluate(inputExpression, parameters);
}
catch (PathEvaluationError e) {
return handleSpecialCase(errorBehavior, errorDefault, e); // TODO by spec, we should cast the defaults only if they are used
return handleSpecialCase(errorBehavior, errorDefault, () -> e); // TODO by spec, we should cast the defaults only if they are used
}

if (pathResult.isEmpty()) {
return handleSpecialCase(emptyBehavior, emptyDefault, NO_ITEMS);
return handleSpecialCase(emptyBehavior, emptyDefault, () -> new JsonValueResultError("JSON path found no items"));
}

if (pathResult.size() > 1) {
return handleSpecialCase(errorBehavior, errorDefault, MULTIPLE_ITEMS);
return handleSpecialCase(errorBehavior, errorDefault, () -> new JsonValueResultError("JSON path found multiple items"));
}

Object item = getOnlyElement(pathResult);
Expand All @@ -289,10 +285,10 @@ public static Object jsonValue(
itemValue = getTypedValue((JsonNode) item);
}
catch (JsonLiteralConversionError e) {
return handleSpecialCase(errorBehavior, errorDefault, new JsonValueResultError("JSON path found an item that cannot be converted to an SQL value", e));
return handleSpecialCase(errorBehavior, errorDefault, () -> new JsonValueResultError("JSON path found an item that cannot be converted to an SQL value", e));
}
if (itemValue.isEmpty()) {
return handleSpecialCase(errorBehavior, errorDefault, INCONVERTIBLE_ITEM);
return handleSpecialCase(errorBehavior, errorDefault, () -> new JsonValueResultError("JSON path found an item that cannot be converted to an SQL value"));
}
typedValue = itemValue.get();
}
Expand All @@ -307,7 +303,7 @@ public static Object jsonValue(
coercion = metadata.getCoercion(((FullConnectorSession) session).getSession(), typedValue.getType(), returnType);
}
catch (OperatorNotFoundException e) {
return handleSpecialCase(errorBehavior, errorDefault, new JsonValueResultError(format(
return handleSpecialCase(errorBehavior, errorDefault, () -> new JsonValueResultError(format(
"Cannot cast value of type %s to declared return type of function JSON_VALUE: %s",
typedValue.getType(),
returnType)));
Expand All @@ -316,20 +312,20 @@ public static Object jsonValue(
return new InterpretedFunctionInvoker(functionManager).invoke(coercion, session, ImmutableList.of(typedValue.getValueAsObject()));
}
catch (RuntimeException e) {
return handleSpecialCase(errorBehavior, errorDefault, new JsonValueResultError(format(
return handleSpecialCase(errorBehavior, errorDefault, () -> new JsonValueResultError(format(
"Cannot cast value of type %s to declared return type of function JSON_VALUE: %s",
typedValue.getType(),
returnType)));
}
}

private static Object handleSpecialCase(long behavior, Object defaultValue, TrinoException error)
private static Object handleSpecialCase(long behavior, Object defaultValue, Supplier<TrinoException> error)
{
switch (EmptyOrErrorBehavior.values()[(int) behavior]) {
case NULL:
return null;
case ERROR:
throw error;
throw error.get();
case DEFAULT:
return defaultValue;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,6 @@ public class TestQueryStateMachine
{
private static final String QUERY = "sql";
private static final URI LOCATION = URI.create("fake://fake-query");
private static final SQLException FAILED_CAUSE = new SQLException("FAILED");
private static final List<Input> INPUTS = ImmutableList.of(new Input(
"connector",
"schema",
Expand Down Expand Up @@ -179,7 +178,7 @@ public void testQueued()
tryGetFutureValue(stateMachine.getStateChange(FINISHING), 2, SECONDS);
});

assertAllTimeSpentInQueueing(FAILED, stateMachine -> stateMachine.transitionToFailed(FAILED_CAUSE));
assertAllTimeSpentInQueueing(FAILED, stateMachine -> stateMachine.transitionToFailed(newFailedCause()));
}

private void assertAllTimeSpentInQueueing(QueryState expectedState, Consumer<QueryStateMachine> stateTransition)
Expand Down Expand Up @@ -230,8 +229,8 @@ public void testPlanning()

stateMachine = createQueryStateMachine();
stateMachine.transitionToPlanning();
assertTrue(stateMachine.transitionToFailed(FAILED_CAUSE));
assertState(stateMachine, FAILED, FAILED_CAUSE);
assertTrue(stateMachine.transitionToFailed(newFailedCause()));
assertState(stateMachine, FAILED, newFailedCause());
}

@Test
Expand Down Expand Up @@ -262,8 +261,8 @@ public void testStarting()

stateMachine = createQueryStateMachine();
stateMachine.transitionToStarting();
assertTrue(stateMachine.transitionToFailed(FAILED_CAUSE));
assertState(stateMachine, FAILED, FAILED_CAUSE);
assertTrue(stateMachine.transitionToFailed(newFailedCause()));
assertState(stateMachine, FAILED, newFailedCause());
}

@Test
Expand Down Expand Up @@ -292,8 +291,8 @@ public void testRunning()

stateMachine = createQueryStateMachine();
stateMachine.transitionToRunning();
assertTrue(stateMachine.transitionToFailed(FAILED_CAUSE));
assertState(stateMachine, FAILED, FAILED_CAUSE);
assertTrue(stateMachine.transitionToFailed(newFailedCause()));
assertState(stateMachine, FAILED, newFailedCause());
}

@Test
Expand All @@ -311,8 +310,8 @@ public void testFinished()
public void testFailed()
{
QueryStateMachine stateMachine = createQueryStateMachine();
assertTrue(stateMachine.transitionToFailed(FAILED_CAUSE));
assertFinalState(stateMachine, FAILED, FAILED_CAUSE);
assertTrue(stateMachine.transitionToFailed(newFailedCause()));
assertFinalState(stateMachine, FAILED, newFailedCause());
}

@Test
Expand Down Expand Up @@ -431,7 +430,7 @@ private static void assertFinalState(QueryStateMachine stateMachine, QueryState
assertFalse(stateMachine.transitionToFinishing());
assertState(stateMachine, expectedState, expectedException);

assertFalse(stateMachine.transitionToFailed(FAILED_CAUSE));
assertFalse(stateMachine.transitionToFailed(newFailedCause()));
assertState(stateMachine, expectedState, expectedException);

// attempt to fail with another exception, which will fail
Expand Down Expand Up @@ -567,4 +566,9 @@ private static void assertEqualSessionsWithoutTransactionId(Session actual, Sess
assertEquals(actual.getSystemProperties(), expected.getSystemProperties());
assertEquals(actual.getCatalogProperties(), expected.getCatalogProperties());
}

private static SQLException newFailedCause()
{
return new SQLException("FAILED");
}
}
1 change: 1 addition & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -2297,6 +2297,7 @@
-Xep:Overrides:ERROR \
-Xep:PreferredInterfaceType:OFF <!-- flags List fields even if initialized with ImmutableList --> \
-Xep:PrimitiveArrayPassedToVarargsMethod:ERROR \
-Xep:StaticAssignmentOfThrowable:ERROR \
-Xep:StreamResourceLeak:ERROR \
-Xep:UnnecessaryLambda:OFF <!-- we deliberately use it in a couple of places --> \
-Xep:UnnecessaryMethodReference:ERROR \
Expand Down

0 comments on commit 1389246

Please sign in to comment.