From 9c13a6a8526828efe69bc18bed5ae37b82372bee Mon Sep 17 00:00:00 2001 From: Neerad Somanchi Date: Wed, 24 Apr 2024 13:51:20 -0700 Subject: [PATCH] Revert "Preserve case for RowType's field name and JSON content when ``CAST``" This reverts commit 5b7cda2e8ff94fff3c96514a90f9ee5f34cbf3f0. --- .../InvalidTypeDefinitionException.java | 39 ------ .../function/SqlFunctionProperties.java | 35 +----- .../facebook/presto/common/type/RowType.java | 5 - .../presto/common/type/TypeSignature.java | 21 +--- .../src/main/sphinx/admin/properties.rst | 13 -- .../src/main/sphinx/functions/json.rst | 24 ---- .../java/com/facebook/presto/Session.java | 3 - .../presto/SystemSessionProperties.java | 6 - .../operator/scalar/JsonToArrayCast.java | 2 +- .../presto/operator/scalar/JsonToMapCast.java | 4 +- .../presto/operator/scalar/JsonToRowCast.java | 11 +- .../presto/sql/analyzer/FeaturesConfig.java | 13 -- .../com/facebook/presto/util/Failures.java | 6 - .../com/facebook/presto/util/JsonUtil.java | 101 +++++----------- .../scalar/AbstractTestFunctions.java | 12 +- .../operator/scalar/FunctionAssertions.java | 48 +++----- .../sql/analyzer/TestFeaturesConfig.java | 7 +- .../presto/type/TestRowOperators.java | 113 +----------------- .../com/facebook/presto/sql/tree/Cast.java | 31 +---- .../presto/spi/StandardErrorCode.java | 1 - 20 files changed, 76 insertions(+), 419 deletions(-) delete mode 100644 presto-common/src/main/java/com/facebook/presto/common/InvalidTypeDefinitionException.java diff --git a/presto-common/src/main/java/com/facebook/presto/common/InvalidTypeDefinitionException.java b/presto-common/src/main/java/com/facebook/presto/common/InvalidTypeDefinitionException.java deleted file mode 100644 index b3b417ef4e59..000000000000 --- a/presto-common/src/main/java/com/facebook/presto/common/InvalidTypeDefinitionException.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.common; - -public class InvalidTypeDefinitionException - extends RuntimeException -{ - public InvalidTypeDefinitionException(String message) - { - this(message, null); - } - - public InvalidTypeDefinitionException(Throwable throwable) - { - this(null, throwable); - } - - public InvalidTypeDefinitionException(String message, Throwable cause) - { - super(message, cause); - } - - @Override - public String getMessage() - { - return super.getMessage(); - } -} diff --git a/presto-common/src/main/java/com/facebook/presto/common/function/SqlFunctionProperties.java b/presto-common/src/main/java/com/facebook/presto/common/function/SqlFunctionProperties.java index 17dc78482061..ce348dbbeac1 100644 --- a/presto-common/src/main/java/com/facebook/presto/common/function/SqlFunctionProperties.java +++ b/presto-common/src/main/java/com/facebook/presto/common/function/SqlFunctionProperties.java @@ -34,7 +34,6 @@ public class SqlFunctionProperties private final Locale sessionLocale; private final String sessionUser; private final boolean fieldNamesInJsonCastEnabled; - private final boolean legacyJsonCast; private final Map extraCredentials; private SqlFunctionProperties( @@ -47,7 +46,6 @@ private SqlFunctionProperties( Locale sessionLocale, String sessionUser, boolean fieldNamesInJsonCastEnabled, - boolean legacyJsonCast, Map extraCredentials) { this.parseDecimalLiteralAsDouble = parseDecimalLiteralAsDouble; @@ -59,7 +57,6 @@ private SqlFunctionProperties( this.sessionLocale = requireNonNull(sessionLocale, "sessionLocale is null"); this.sessionUser = requireNonNull(sessionUser, "sessionUser is null"); this.fieldNamesInJsonCastEnabled = fieldNamesInJsonCastEnabled; - this.legacyJsonCast = legacyJsonCast; this.extraCredentials = requireNonNull(extraCredentials, "extraCredentials is null"); } @@ -114,11 +111,6 @@ public boolean isFieldNamesInJsonCastEnabled() return fieldNamesInJsonCastEnabled; } - public boolean isLegacyJsonCast() - { - return legacyJsonCast; - } - @Override public boolean equals(Object o) { @@ -137,16 +129,13 @@ public boolean equals(Object o) Objects.equals(sessionStartTime, that.sessionStartTime) && Objects.equals(sessionLocale, that.sessionLocale) && Objects.equals(sessionUser, that.sessionUser) && - Objects.equals(extraCredentials, that.extraCredentials) && - Objects.equals(legacyJsonCast, that.legacyJsonCast); + Objects.equals(extraCredentials, that.extraCredentials); } @Override public int hashCode() { - return Objects.hash(parseDecimalLiteralAsDouble, legacyRowFieldOrdinalAccessEnabled, timeZoneKey, - legacyTimestamp, legacyMapSubscript, sessionStartTime, sessionLocale, sessionUser, - extraCredentials, legacyJsonCast); + return Objects.hash(parseDecimalLiteralAsDouble, legacyRowFieldOrdinalAccessEnabled, timeZoneKey, legacyTimestamp, legacyMapSubscript, sessionStartTime, sessionLocale, sessionUser, extraCredentials); } public static Builder builder() @@ -165,7 +154,6 @@ public static class Builder private Locale sessionLocale; private String sessionUser; private boolean fieldNamesInJsonCastEnabled; - private boolean legacyJsonCast; private Map extraCredentials = emptyMap(); private Builder() {} @@ -230,26 +218,9 @@ public Builder setFieldNamesInJsonCastEnabled(boolean fieldNamesInJsonCastEnable return this; } - public Builder setLegacyJsonCast(boolean legacyJsonCast) - { - this.legacyJsonCast = legacyJsonCast; - return this; - } - public SqlFunctionProperties build() { - return new SqlFunctionProperties( - parseDecimalLiteralAsDouble, - legacyRowFieldOrdinalAccessEnabled, - timeZoneKey, - legacyTimestamp, - legacyMapSubscript, - sessionStartTime, - sessionLocale, - sessionUser, - fieldNamesInJsonCastEnabled, - legacyJsonCast, - extraCredentials); + return new SqlFunctionProperties(parseDecimalLiteralAsDouble, legacyRowFieldOrdinalAccessEnabled, timeZoneKey, legacyTimestamp, legacyMapSubscript, sessionStartTime, sessionLocale, sessionUser, fieldNamesInJsonCastEnabled, extraCredentials); } } } diff --git a/presto-common/src/main/java/com/facebook/presto/common/type/RowType.java b/presto-common/src/main/java/com/facebook/presto/common/type/RowType.java index 347665bba084..0740f0ebf6b6 100644 --- a/presto-common/src/main/java/com/facebook/presto/common/type/RowType.java +++ b/presto-common/src/main/java/com/facebook/presto/common/type/RowType.java @@ -83,11 +83,6 @@ public static Field field(Type type) return new Field(Optional.empty(), type); } - public static Field field(String name, Type type, boolean delimited) - { - return new Field(Optional.of(name), type, delimited); - } - private static TypeSignature makeSignature(List fields) { int size = fields.size(); diff --git a/presto-common/src/main/java/com/facebook/presto/common/type/TypeSignature.java b/presto-common/src/main/java/com/facebook/presto/common/type/TypeSignature.java index 4a6649873436..5474b2dc0f5a 100644 --- a/presto-common/src/main/java/com/facebook/presto/common/type/TypeSignature.java +++ b/presto-common/src/main/java/com/facebook/presto/common/type/TypeSignature.java @@ -16,7 +16,6 @@ import com.facebook.drift.annotations.ThriftConstructor; import com.facebook.drift.annotations.ThriftField; import com.facebook.drift.annotations.ThriftStruct; -import com.facebook.presto.common.InvalidTypeDefinitionException; import com.facebook.presto.common.QualifiedObjectName; import com.facebook.presto.common.type.BigintEnumType.LongEnumMap; import com.facebook.presto.common.type.VarcharEnumType.VarcharEnumMap; @@ -511,7 +510,6 @@ private static TypeSignature parseRowTypeSignature(String signature, Set List fields = new ArrayList<>(); - Set distinctFieldNames = new HashSet<>(); for (int i = StandardTypes.ROW.length() + 1; i < signature.length(); i++) { char c = signature.charAt(i); switch (state) { @@ -558,19 +556,13 @@ else if (c == ')' && bracketLevel > 1) { } else if (c == ')') { verify(tokenStart >= 0, "Expect tokenStart to be non-negative"); - TypeSignatureParameter parameter = parseTypeOrNamedType(signature.substring(tokenStart, i).trim(), literalParameters); - parameter.getNamedTypeSignature().getName() - .ifPresent(fieldName -> checkDuplicateAndAdd(distinctFieldNames, fieldName)); - fields.add(parameter); + fields.add(parseTypeOrNamedType(signature.substring(tokenStart, i).trim(), literalParameters)); tokenStart = -1; state = RowTypeSignatureParsingState.FINISHED; } else if (c == ',' && bracketLevel == 1) { verify(tokenStart >= 0, "Expect tokenStart to be non-negative"); - TypeSignatureParameter parameter = parseTypeOrNamedType(signature.substring(tokenStart, i).trim(), literalParameters); - parameter.getNamedTypeSignature().getName() - .ifPresent(fieldName -> checkDuplicateAndAdd(distinctFieldNames, fieldName)); - fields.add(parameter); + fields.add(parseTypeOrNamedType(signature.substring(tokenStart, i).trim(), literalParameters)); tokenStart = -1; state = RowTypeSignatureParsingState.START_OF_FIELD; } @@ -586,7 +578,6 @@ else if (c == ')' && bracketLevel > 1) { else if (c == ')') { verify(tokenStart >= 0, "Expect tokenStart to be non-negative"); verify(delimitedColumnName != null, "Expect delimitedColumnName to be non-null"); - checkDuplicateAndAdd(distinctFieldNames, delimitedColumnName); fields.add(TypeSignatureParameter.of(new NamedTypeSignature( Optional.of(new RowFieldName(delimitedColumnName, true)), parseTypeSignature(signature.substring(tokenStart, i).trim(), literalParameters)))); @@ -597,7 +588,6 @@ else if (c == ')') { else if (c == ',' && bracketLevel == 1) { verify(tokenStart >= 0, "Expect tokenStart to be non-negative"); verify(delimitedColumnName != null, "Expect delimitedColumnName to be non-null"); - checkDuplicateAndAdd(distinctFieldNames, delimitedColumnName); fields.add(TypeSignatureParameter.of(new NamedTypeSignature( Optional.of(new RowFieldName(delimitedColumnName, true)), parseTypeSignature(signature.substring(tokenStart, i).trim(), literalParameters)))); @@ -619,13 +609,6 @@ else if (c == ',' && bracketLevel == 1) { return new TypeSignature(signature.substring(0, StandardTypes.ROW.length()), fields); } - private static void checkDuplicateAndAdd(Set fieldNames, String fieldName) - { - if (!fieldNames.add(fieldName)) { - throw new InvalidTypeDefinitionException("Duplicate field: " + fieldName); - } - } - private static TypeSignatureParameter parseTypeOrNamedType(String typeOrNamedType, Set literalParameters) { int split = typeOrNamedType.indexOf(' '); diff --git a/presto-docs/src/main/sphinx/admin/properties.rst b/presto-docs/src/main/sphinx/admin/properties.rst index 7b4c94ec302b..0d1bce55ac0f 100644 --- a/presto-docs/src/main/sphinx/admin/properties.rst +++ b/presto-docs/src/main/sphinx/admin/properties.rst @@ -989,16 +989,3 @@ Logging Properties * **Default value:** ``100MB`` The maximum file size for the log file of the HTTP server. - - -Legacy Compatible Properties ------------------------------- - -``legacy_json_cast`` -^^^^^^^^^^^^^^^^^^^^^ - - * **Type:** ``boolean`` - * **Default value:** ``true`` - - When casting from ``JSON`` to ``ROW``, ignore the case of field names in ``RowType`` for legacy support so that the matching is case-insensitive. - Set ``legacy_json_cast`` to ``false`` to strictly enforce the case-sensitivity of double quoted field names in ``RowType`` when matching. Matching for unquoted field names remains case-insensitive. \ No newline at end of file diff --git a/presto-docs/src/main/sphinx/functions/json.rst b/presto-docs/src/main/sphinx/functions/json.rst index c51338880464..28be7c0d9006 100644 --- a/presto-docs/src/main/sphinx/functions/json.rst +++ b/presto-docs/src/main/sphinx/functions/json.rst @@ -71,30 +71,6 @@ Cast from JSON SELECT CAST(JSON '{"k1": [1, 23], "k2": 456}' AS MAP(VARCHAR, JSON)); -- {k1 = JSON '[1,23]', k2 = JSON '456'} SELECT CAST(JSON '[null]' AS ARRAY(JSON)); -- [JSON 'null'] -.. note:: - - When casting from ``JSON`` to ``ROW``, for legacy support the case of double quoted field names - in ``RowType`` is ignored when matching. For example:: - - SELECT CAST(JSON '{"v1":123,"V2":"abc","v3":true}' AS ROW(v1 BIGINT, v2 VARCHAR, v3 BOOLEAN)); -- {v1=123, v2=abc, v3=true} - SELECT CAST(JSON '{"v1":123,"V2":"abc","v3":true}' AS ROW(v1 BIGINT, "V2" VARCHAR, "V3" BOOLEAN)); -- {v1=123, V2=abc, V3=true} - - The following statement returns an error due to duplicate field:: - - SELECT CAST(JSON '{"v1":123,"V2":"abc","v2":"abc2","v3":true}' AS ROW(v1 BIGINT, "V2" VARCHAR, v2 VARCHAR, "V3" BOOLEAN)); - - To enforce the case of field names in ``RowType`` when casting from ``JSON`` to ``ROW``, set the configuration property ``legacy_json_cast`` to ``false`` - in the coordinator and the worker's `configuration properties <../admin/properties.html#legacy-compatible-properties>`_. - After setting the property, the matching is case-sensitive for double quoted field names and - remains case-insensitive for unquoted field names. For example:: - - SELECT CAST(JSON '{"v1":123,"V2":"abc","v3":true}' AS ROW(v1 BIGINT, v2 VARCHAR, v3 BOOLEAN)); -- {v1=123, v2=abc, v3=true} - SELECT CAST(JSON '{"v1":123,"V2":"abc","v3":true}' AS ROW(v1 BIGINT, "V2" VARCHAR, v3 BOOLEAN)); -- {v1=123, V2=abc, v3=true} - SELECT CAST(JSON '{"v1":123,"V2":"abc","v3":true}' AS ROW(v1 BIGINT, "v2" VARCHAR, v3 BOOLEAN)); -- {v1=123, v2=null, v3=true} - SELECT CAST(JSON '{"v1":123,"V2":"abc", "v2":"abc2","v3":true}' AS ROW(v1 BIGINT, v2 VARCHAR, "V2" VARCHAR, v3 BOOLEAN)); -- {v1=123, v2=abc2, V2=abc, v3=true} - - If the name of a field does not match (including case sensitivity), the value is ``null``. - .. note:: When casting from ``JSON`` to ``ROW``, both JSON array and JSON object are supported. JSON Functions diff --git a/presto-main/src/main/java/com/facebook/presto/Session.java b/presto-main/src/main/java/com/facebook/presto/Session.java index d775c7368bb5..6d343e9afda3 100644 --- a/presto-main/src/main/java/com/facebook/presto/Session.java +++ b/presto-main/src/main/java/com/facebook/presto/Session.java @@ -52,7 +52,6 @@ import java.util.TimeZone; import java.util.stream.Collectors; -import static com.facebook.presto.SystemSessionProperties.LEGACY_JSON_CAST; import static com.facebook.presto.SystemSessionProperties.isFieldNameInJsonCastEnabled; import static com.facebook.presto.SystemSessionProperties.isLegacyMapSubscript; import static com.facebook.presto.SystemSessionProperties.isLegacyRowFieldOrdinalAccessEnabled; @@ -497,7 +496,6 @@ public ConnectorSession toConnectorSession() public SqlFunctionProperties getSqlFunctionProperties() { - boolean legacyJsonCast = this.sessionPropertyManager.decodeSystemPropertyValue(LEGACY_JSON_CAST, null, Boolean.class); return SqlFunctionProperties.builder() .setTimeZoneKey(timeZoneKey) .setLegacyRowFieldOrdinalAccessEnabled(isLegacyRowFieldOrdinalAccessEnabled(this)) @@ -508,7 +506,6 @@ public SqlFunctionProperties getSqlFunctionProperties() .setSessionLocale(getLocale()) .setSessionUser(getUser()) .setFieldNamesInJsonCastEnabled(isFieldNameInJsonCastEnabled(this)) - .setLegacyJsonCast(legacyJsonCast) .setExtraCredentials(identity.getExtraCredentials()) .build(); } diff --git a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java index 64c0fc34b30b..8d6e2173661a 100644 --- a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java +++ b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java @@ -311,7 +311,6 @@ public final class SystemSessionProperties public static final String ADD_PARTIAL_NODE_FOR_ROW_NUMBER_WITH_LIMIT = "add_partial_node_for_row_number_with_limit"; public static final String REWRITE_CASE_TO_MAP_ENABLED = "rewrite_case_to_map_enabled"; public static final String FIELD_NAMES_IN_JSON_CAST_ENABLED = "field_names_in_json_cast_enabled"; - public static final String LEGACY_JSON_CAST = "legacy_json_cast"; public static final String PULL_EXPRESSION_FROM_LAMBDA_ENABLED = "pull_expression_from_lambda_enabled"; public static final String REWRITE_CONSTANT_ARRAY_CONTAINS_TO_IN_EXPRESSION = "rewrite_constant_array_contains_to_in_expression"; public static final String INFER_INEQUALITY_PREDICATES = "infer_inequality_predicates"; @@ -1754,11 +1753,6 @@ public SystemSessionProperties( "Include field names in json output when casting rows", featuresConfig.isFieldNamesInJsonCastEnabled(), false), - booleanProperty( - LEGACY_JSON_CAST, - "Keep the legacy json cast behavior, do not reserve the case for field names when casting to row type", - featuresConfig.isLegacyJsonCast(), - false), booleanProperty( OPTIMIZE_JOIN_PROBE_FOR_EMPTY_BUILD_RUNTIME, "Optimize join probe at runtime if build side is empty", diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToArrayCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToArrayCast.java index fbffb1d1ef6b..3b520dfc8c0c 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToArrayCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToArrayCast.java @@ -95,7 +95,7 @@ public static Block toArray(ArrayType arrayType, BlockBuilderAppender elementApp } BlockBuilder blockBuilder = arrayType.getElementType().createBlockBuilder(null, 20); while (jsonParser.nextToken() != JsonToken.END_ARRAY) { - elementAppender.append(jsonParser, blockBuilder, properties); + elementAppender.append(jsonParser, blockBuilder); } if (jsonParser.nextToken() != null) { throw new JsonCastException(format("Unexpected trailing token: %s", jsonParser.getText())); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToMapCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToMapCast.java index 54ad54a11bc0..909d3d430d55 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToMapCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToMapCast.java @@ -103,9 +103,9 @@ public static Block toMap(MapType mapType, BlockBuilderAppender keyAppender, Blo HashTable hashTable = new HashTable(mapType.getKeyType(), singleMapBlockBuilder); int position = 0; while (jsonParser.nextToken() != JsonToken.END_OBJECT) { - keyAppender.append(jsonParser, singleMapBlockBuilder, properties); + keyAppender.append(jsonParser, singleMapBlockBuilder); jsonParser.nextToken(); - valueAppender.append(jsonParser, singleMapBlockBuilder, properties); + valueAppender.append(jsonParser, singleMapBlockBuilder); // Duplicate key detection is required even if the JSON is valid. // For example: CAST(JSON '{"1": 1, "01": 2}' AS MAP). diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToRowCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToRowCast.java index bb87a4285528..ab7c0d37e681 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToRowCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToRowCast.java @@ -35,6 +35,8 @@ import java.lang.invoke.MethodHandle; import java.util.List; +import java.util.Map; +import java.util.Optional; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.ArgumentProperty.valueTypeArgumentProperty; @@ -46,6 +48,7 @@ import static com.facebook.presto.util.JsonUtil.JSON_FACTORY; import static com.facebook.presto.util.JsonUtil.canCastFromJson; import static com.facebook.presto.util.JsonUtil.createJsonParser; +import static com.facebook.presto.util.JsonUtil.getFieldNameToIndex; import static com.facebook.presto.util.JsonUtil.parseJsonToSingleRowBlock; import static com.facebook.presto.util.JsonUtil.truncateIfNecessaryForErrorMessage; import static com.facebook.presto.util.Reflection.methodHandle; @@ -58,7 +61,7 @@ public class JsonToRowCast extends SqlOperator { public static final JsonToRowCast JSON_TO_ROW = new JsonToRowCast(); - private static final MethodHandle METHOD_HANDLE = methodHandle(JsonToRowCast.class, "toRow", RowType.class, BlockBuilderAppender[].class, SqlFunctionProperties.class, Slice.class); + private static final MethodHandle METHOD_HANDLE = methodHandle(JsonToRowCast.class, "toRow", RowType.class, BlockBuilderAppender[].class, Optional.class, SqlFunctionProperties.class, Slice.class); private JsonToRowCast() { @@ -80,7 +83,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab BlockBuilderAppender[] fieldAppenders = rowFields.stream() .map(rowField -> createBlockBuilderAppender(rowField.getType())) .toArray(BlockBuilderAppender[]::new); - MethodHandle methodHandle = METHOD_HANDLE.bindTo(rowType).bindTo(fieldAppenders); + MethodHandle methodHandle = METHOD_HANDLE.bindTo(rowType).bindTo(fieldAppenders).bindTo(getFieldNameToIndex(rowFields)); return new BuiltInScalarFunctionImplementation( true, ImmutableList.of(valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), @@ -91,6 +94,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab public static Block toRow( RowType rowType, BlockBuilderAppender[] fieldAppenders, + Optional> fieldNameToIndex, SqlFunctionProperties properties, Slice json) { @@ -109,8 +113,7 @@ public static Block toRow( jsonParser, (SingleRowBlockWriter) rowBlockBuilder.beginBlockEntry(), fieldAppenders, - rowType, - properties); + fieldNameToIndex); rowBlockBuilder.closeEntry(); if (jsonParser.nextToken() != null) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java index 95aac7243cbb..f7e4fcfe6a7e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java @@ -277,7 +277,6 @@ public class FeaturesConfig private boolean useDefaultsForCorrelatedAggregationPushdownThroughOuterJoins = true; private boolean mergeDuplicateAggregationsEnabled = true; private boolean fieldNamesInJsonCastEnabled; - private boolean legacyJsonCast = true; private boolean mergeAggregationsWithAndWithoutFilter; private boolean simplifyPlanWithEmptyInput = true; private PushDownFilterThroughCrossJoinStrategy pushDownFilterExpressionEvaluationThroughCrossJoin = PushDownFilterThroughCrossJoinStrategy.REWRITTEN_TO_INNER_JOIN; @@ -655,18 +654,6 @@ public FeaturesConfig setFieldNamesInJsonCastEnabled(boolean fieldNamesInJsonCas return this; } - public boolean isLegacyJsonCast() - { - return legacyJsonCast; - } - - @Config("legacy-json-cast") - public FeaturesConfig setLegacyJsonCast(boolean legacyJsonCast) - { - this.legacyJsonCast = legacyJsonCast; - return this; - } - @Config("reduce-agg-for-complex-types-enabled") public FeaturesConfig setReduceAggForComplexTypesEnabled(boolean reduceAggForComplexTypesEnabled) { diff --git a/presto-main/src/main/java/com/facebook/presto/util/Failures.java b/presto-main/src/main/java/com/facebook/presto/util/Failures.java index 0d8ece16f113..de44e9fc3b7b 100644 --- a/presto-main/src/main/java/com/facebook/presto/util/Failures.java +++ b/presto-main/src/main/java/com/facebook/presto/util/Failures.java @@ -16,7 +16,6 @@ import com.facebook.presto.ExceededMemoryLimitException; import com.facebook.presto.client.ErrorLocation; import com.facebook.presto.common.ErrorCode; -import com.facebook.presto.common.InvalidTypeDefinitionException; import com.facebook.presto.execution.ExecutionFailureInfo; import com.facebook.presto.execution.Failure; import com.facebook.presto.spi.ErrorCause; @@ -41,7 +40,6 @@ import static com.facebook.presto.spi.ErrorCause.UNKNOWN; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; -import static com.facebook.presto.spi.StandardErrorCode.INVALID_TYPE_DEFINITION; import static com.facebook.presto.spi.StandardErrorCode.SLICE_TOO_LARGE; import static com.facebook.presto.spi.StandardErrorCode.SYNTAX_ERROR; import static com.google.common.base.Functions.toStringFunction; @@ -173,10 +171,6 @@ private static ErrorCode toErrorCode(Throwable throwable) return SLICE_TOO_LARGE.toErrorCode(); } - if (throwable instanceof InvalidTypeDefinitionException) { - return INVALID_TYPE_DEFINITION.toErrorCode(); - } - if (throwable instanceof PrestoException) { return ((PrestoException) throwable).getErrorCode(); } diff --git a/presto-main/src/main/java/com/facebook/presto/util/JsonUtil.java b/presto-main/src/main/java/com/facebook/presto/util/JsonUtil.java index 4b48dcbd8fff..edc9765cdd0c 100644 --- a/presto-main/src/main/java/com/facebook/presto/util/JsonUtil.java +++ b/presto-main/src/main/java/com/facebook/presto/util/JsonUtil.java @@ -54,6 +54,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.TreeMap; @@ -95,7 +96,6 @@ import static java.lang.String.format; import static java.math.RoundingMode.HALF_UP; import static java.nio.charset.StandardCharsets.UTF_8; -import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; public final class JsonUtil @@ -914,7 +914,7 @@ private static BigDecimal currentTokenAsJavaDecimal(JsonParser parser, int preci // given a JSON parser, write to the BlockBuilder public interface BlockBuilderAppender { - void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException; static BlockBuilderAppender createBlockBuilderAppender(Type type) @@ -955,7 +955,7 @@ static BlockBuilderAppender createBlockBuilderAppender(Type type) case StandardTypes.VARCHAR: return new VarcharBlockBuilderAppender(type); case StandardTypes.JSON: - return (parser, blockBuilder, sqlFunctionProperties) -> { + return (parser, blockBuilder) -> { String json = OBJECT_MAPPED_UNORDERED.writeValueAsString(parser.readValueAsTree()); JSON.writeSlice(blockBuilder, Slices.utf8Slice(json)); }; @@ -974,7 +974,7 @@ static BlockBuilderAppender createBlockBuilderAppender(Type type) for (int i = 0; i < fieldAppenders.length; i++) { fieldAppenders[i] = createBlockBuilderAppender(rowFields.get(i).getType()); } - return new RowBlockBuilderAppender(fieldAppenders, rowType); + return new RowBlockBuilderAppender(fieldAppenders, getFieldNameToIndex(rowFields)); default: throw new PrestoException(INVALID_FUNCTION_ARGUMENT, format("Unsupported type: %s", type)); } @@ -985,7 +985,7 @@ private static class BooleanBlockBuilderAppender implements BlockBuilderAppender { @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Boolean result = currentTokenAsBoolean(parser); @@ -1002,7 +1002,7 @@ private static class TinyintBlockBuilderAppender implements BlockBuilderAppender { @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Long result = currentTokenAsTinyint(parser); @@ -1019,7 +1019,7 @@ private static class SmallintBlockBuilderAppender implements BlockBuilderAppender { @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Long result = currentTokenAsInteger(parser); @@ -1036,7 +1036,7 @@ private static class IntegerBlockBuilderAppender implements BlockBuilderAppender { @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Long result = currentTokenAsInteger(parser); @@ -1053,7 +1053,7 @@ private static class BigintBlockBuilderAppender implements BlockBuilderAppender { @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Long result = currentTokenAsBigint(parser); @@ -1070,7 +1070,7 @@ private static class RealBlockBuilderAppender implements BlockBuilderAppender { @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Long result = currentTokenAsReal(parser); @@ -1087,7 +1087,7 @@ private static class DoubleBlockBuilderAppender implements BlockBuilderAppender { @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Double result = currentTokenAsDouble(parser); @@ -1111,7 +1111,7 @@ private static class ShortDecimalBlockBuilderAppender } @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Long result = currentTokenAsShortDecimal(parser, type.getPrecision(), type.getScale()); @@ -1136,7 +1136,7 @@ private static class LongDecimalBlockBuilderAppender } @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Slice result = currentTokenAsLongDecimal(parser, type.getPrecision(), type.getScale()); @@ -1161,7 +1161,7 @@ private static class VarcharBlockBuilderAppender } @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { Slice result = currentTokenAsVarchar(parser); @@ -1185,7 +1185,7 @@ private static class ArrayBlockBuilderAppender } @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { if (parser.getCurrentToken() == JsonToken.VALUE_NULL) { @@ -1198,7 +1198,7 @@ public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProp } BlockBuilder entryBuilder = blockBuilder.beginBlockEntry(); while (parser.nextToken() != END_ARRAY) { - elementAppender.append(parser, entryBuilder, sqlFunctionProperties); + elementAppender.append(parser, entryBuilder); } blockBuilder.closeEntry(); } @@ -1219,7 +1219,7 @@ private static class MapBlockBuilderAppender } @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { if (parser.getCurrentToken() == JsonToken.VALUE_NULL) { @@ -1234,9 +1234,9 @@ public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProp HashTable entryBuilderHashTable = new HashTable(keyType, entryBuilder); int position = 0; while (parser.nextToken() != END_OBJECT) { - keyAppender.append(parser, entryBuilder, sqlFunctionProperties); + keyAppender.append(parser, entryBuilder); parser.nextToken(); - valueAppender.append(parser, entryBuilder, sqlFunctionProperties); + valueAppender.append(parser, entryBuilder); if (!entryBuilderHashTable.addIfAbsent(position)) { throw new JsonCastException("Duplicate keys are not allowed"); } @@ -1250,18 +1250,16 @@ private static class RowBlockBuilderAppender implements BlockBuilderAppender { final BlockBuilderAppender[] fieldAppenders; - final RowType rowType; + final Optional> fieldNameToIndex; - RowBlockBuilderAppender( - BlockBuilderAppender[] fieldAppenders, - RowType rowType) + RowBlockBuilderAppender(BlockBuilderAppender[] fieldAppenders, Optional> fieldNameToIndex) { this.fieldAppenders = fieldAppenders; - this.rowType = rowType; + this.fieldNameToIndex = fieldNameToIndex; } @Override - public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProperties sqlFunctionProperties) + public void append(JsonParser parser, BlockBuilder blockBuilder) throws IOException { if (parser.getCurrentToken() == JsonToken.VALUE_NULL) { @@ -1277,21 +1275,20 @@ public void append(JsonParser parser, BlockBuilder blockBuilder, SqlFunctionProp parser, (SingleRowBlockWriter) blockBuilder.beginBlockEntry(), fieldAppenders, - rowType, - sqlFunctionProperties); + fieldNameToIndex); blockBuilder.closeEntry(); } } - public static Optional> getFieldToIndex(List rowFields) + public static Optional> getFieldNameToIndex(List rowFields) { if (!rowFields.get(0).getName().isPresent()) { return Optional.empty(); } - Map fieldNameToIndex = new HashMap<>(rowFields.size()); + Map fieldNameToIndex = new HashMap<>(rowFields.size()); for (int i = 0; i < rowFields.size(); i++) { - fieldNameToIndex.put(rowFields.get(i), i); + fieldNameToIndex.put(rowFields.get(i).getName().get(), i); } return Optional.of(fieldNameToIndex); } @@ -1303,14 +1300,13 @@ public static void parseJsonToSingleRowBlock( JsonParser parser, SingleRowBlockWriter singleRowBlockWriter, BlockBuilderAppender[] fieldAppenders, - RowType rowType, - SqlFunctionProperties sqlFunctionProperties) + Optional> fieldNameToIndex) throws IOException { if (parser.getCurrentToken() == START_ARRAY) { for (int i = 0; i < fieldAppenders.length; i++) { parser.nextToken(); - fieldAppenders[i].append(parser, singleRowBlockWriter, sqlFunctionProperties); + fieldAppenders[i].append(parser, singleRowBlockWriter); } if (parser.nextToken() != JsonToken.END_ARRAY) { throw new JsonCastException(format("Expected json array ending, but got %s", parser.getText())); @@ -1318,47 +1314,18 @@ public static void parseJsonToSingleRowBlock( } else { verify(parser.getCurrentToken() == START_OBJECT); - Optional> fieldToIndex = getFieldToIndex(rowType.getFields()); - if (!fieldToIndex.isPresent()) { + if (!fieldNameToIndex.isPresent()) { throw new JsonCastException("Cannot cast a JSON object to anonymous row type. Input must be a JSON array."); } boolean[] fieldWritten = new boolean[fieldAppenders.length]; int numFieldsWritten = 0; - Map caseSensitiveWhenMatching = new HashMap<>(); - Map caseInsensitiveWhenMatching = new HashMap<>(); - if (sqlFunctionProperties.isLegacyJsonCast()) { - fieldToIndex.get().entrySet().stream() - .forEach(entry -> caseInsensitiveWhenMatching.put( - entry.getKey().getName().get().toLowerCase(ENGLISH), - entry.getValue())); - } - else { - fieldToIndex.get().entrySet().stream() - .forEach(entry -> { - if (entry.getKey().isDelimited()) { - caseSensitiveWhenMatching.put( - entry.getKey().getName().get(), - entry.getValue()); - } - else { - caseInsensitiveWhenMatching.put( - entry.getKey().getName().get().toLowerCase(ENGLISH), - entry.getValue()); - } - }); - } while (parser.nextToken() != JsonToken.END_OBJECT) { if (parser.currentToken() != FIELD_NAME) { throw new JsonCastException(format("Expected a json field name, but got %s", parser.getText())); } - - String fieldName = parser.getText(); - Integer fieldIndex = caseSensitiveWhenMatching.get(fieldName); - if (fieldIndex == null) { - fieldIndex = caseInsensitiveWhenMatching.get(fieldName.toLowerCase(ENGLISH)); - } - + String fieldName = parser.getText().toLowerCase(Locale.ENGLISH); + Integer fieldIndex = fieldNameToIndex.get().get(fieldName); parser.nextToken(); if (fieldIndex != null) { if (fieldWritten[fieldIndex]) { @@ -1366,9 +1333,7 @@ public static void parseJsonToSingleRowBlock( } fieldWritten[fieldIndex] = true; numFieldsWritten++; - fieldAppenders[fieldIndex].append(parser, - singleRowBlockWriter.getFieldBlockBuilder(fieldIndex), - sqlFunctionProperties); + fieldAppenders[fieldIndex].append(parser, singleRowBlockWriter.getFieldBlockBuilder(fieldIndex)); } else { parser.skipChildren(); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/AbstractTestFunctions.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/AbstractTestFunctions.java index ba4db0e159ae..d25bde5ba8f7 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/AbstractTestFunctions.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/AbstractTestFunctions.java @@ -139,6 +139,13 @@ protected void assertDecimalFunction(String statement, SqlDecimal expectedResult expectedResult); } + // this is not safe as it catches all RuntimeExceptions + @Deprecated + protected void assertInvalidFunction(String projection) + { + functionAssertions.assertInvalidFunction(projection); + } + protected void assertInvalidFunction(String projection, StandardErrorCode errorCode, String messagePattern) { functionAssertions.assertInvalidFunction(projection, errorCode, messagePattern); @@ -174,11 +181,6 @@ protected void assertNumericOverflow(String projection, String message) functionAssertions.assertNumericOverflow(projection, message); } - protected void assertInvalidTypeDefinition(String projection, String message) - { - functionAssertions.assertInvalidTypeDefinition(projection, message); - } - protected void assertInvalidCast(String projection) { functionAssertions.assertInvalidCast(projection); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java index 36f27421f066..bf4a2ed33fe9 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java @@ -14,7 +14,6 @@ package com.facebook.presto.operator.scalar; import com.facebook.presto.Session; -import com.facebook.presto.common.InvalidTypeDefinitionException; import com.facebook.presto.common.Page; import com.facebook.presto.common.PageBuilder; import com.facebook.presto.common.Utils; @@ -134,7 +133,6 @@ import static com.facebook.presto.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static com.facebook.presto.spi.StandardErrorCode.INVALID_CAST_ARGUMENT; import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; -import static com.facebook.presto.spi.StandardErrorCode.INVALID_TYPE_DEFINITION; import static com.facebook.presto.spi.StandardErrorCode.NUMERIC_VALUE_OUT_OF_RANGE; import static com.facebook.presto.spi.schedule.NodeSelectionStrategy.HARD_AFFINITY; import static com.facebook.presto.sql.ExpressionUtils.rewriteIdentifiersToSymbolReferences; @@ -145,7 +143,6 @@ import static com.facebook.presto.sql.relational.SqlToRowExpressionTranslator.translate; import static com.facebook.presto.testing.TestingTaskContext.createTaskContext; import static com.facebook.presto.util.AnalyzerUtil.createParsingOptions; -import static com.facebook.presto.util.Failures.toFailure; import static io.airlift.slice.SizeOf.sizeOf; import static io.airlift.units.DataSize.Unit.BYTE; import static java.lang.String.format; @@ -224,22 +221,12 @@ public FunctionAssertions(Session session) public FunctionAssertions(Session session, FeaturesConfig featuresConfig) { - this(session, featuresConfig, false); - } - - public FunctionAssertions(Session session, FeaturesConfig featuresConfig, boolean refreshSession) - { - requireNonNull(session, "session is null"); + this.session = requireNonNull(session, "session is null"); runner = new LocalQueryRunner(session, featuresConfig); - if (refreshSession) { - this.session = runner.getDefaultSession(); - } - else { - this.session = session; - } metadata = runner.getMetadata(); compiler = runner.getExpressionCompiler(); } + public FunctionAndTypeManager getFunctionAndTypeManager() { return runner.getFunctionAndTypeManager(); @@ -354,6 +341,19 @@ private Object selectUniqueValue(String projection, Type expectedType, Session s return Iterables.getOnlyElement(resultSet); } + // this is not safe as it catches all RuntimeExceptions + @Deprecated + public void assertInvalidFunction(String projection) + { + try { + Object value = evaluateInvalid(projection); + fail(format("Expected to throw but got %s", value)); + } + catch (RuntimeException e) { + // Expected + } + } + public void assertInvalidFunction(String projection, StandardErrorCode errorCode, String messagePattern) { try { @@ -412,24 +412,6 @@ public void assertInvalidFunction(String projection, SemanticErrorCode expectedE } } - public void assertInvalidTypeDefinition(String projection, String message) - { - try { - Object value = evaluateInvalid(projection); - fail("Expected to throw an INVALID_CAST_ARGUMENT exception, but got " + value); - } - catch (InvalidTypeDefinitionException e) { - try { - assertEquals(toFailure(e).getErrorCode(), INVALID_TYPE_DEFINITION.toErrorCode()); - assertEquals(e.getMessage(), message); - } - catch (Throwable failure) { - failure.addSuppressed(e); - throw failure; - } - } - } - public void assertInvalidFunction(String projection, ErrorCodeSupplier expectedErrorCode) { try { diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java index 2ab3965ccdf4..1f88b6d878cc 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java @@ -270,8 +270,7 @@ public void testDefaults() .setDefaultWriterReplicationCoefficient(3.0) .setDefaultViewSecurityMode(DEFINER) .setCteHeuristicReplicationThreshold(4) - .setUseHistograms(false) - .setLegacyJsonCast(true)); + .setUseHistograms(false)); } @Test @@ -455,7 +454,6 @@ public void testExplicitPropertyMappings() .put("optimizer.push-aggregation-below-join-byte-reduction-threshold", "0.9") .put("optimizer.prefilter-for-groupby-limit", "true") .put("field-names-in-json-cast-enabled", "true") - .put("legacy-json-cast", "false") .put("optimizer.optimize-probe-for-empty-build-runtime", "true") .put("optimizer.use-defaults-for-correlated-aggregation-pushdown-through-outer-joins", "false") .put("optimizer.merge-duplicate-aggregations", "false") @@ -698,8 +696,7 @@ public void testExplicitPropertyMappings() .setDefaultWriterReplicationCoefficient(5.0) .setDefaultViewSecurityMode(INVOKER) .setCteHeuristicReplicationThreshold(2) - .setUseHistograms(true) - .setLegacyJsonCast(false); + .setUseHistograms(true); assertFullMapping(properties, expected); } diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestRowOperators.java b/presto-main/src/test/java/com/facebook/presto/type/TestRowOperators.java index ff25784446a6..1db892b85df0 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestRowOperators.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestRowOperators.java @@ -19,7 +19,6 @@ import com.facebook.presto.common.type.RowType; import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.Type; -import com.facebook.presto.common.type.VarcharType; import com.facebook.presto.operator.scalar.AbstractTestFunctions; import com.facebook.presto.operator.scalar.FunctionAssertions; import com.facebook.presto.spi.StandardErrorCode; @@ -75,8 +74,6 @@ public class TestRowOperators { private static FunctionAssertions legacyRowFieldOrdinalAccess; private static FunctionAssertions fieldNameInJsonCastEnabled; - private static FunctionAssertions legacyJsonCastEnabled; - private static FunctionAssertions legacyJsonCastDisabled; @BeforeClass public void setUp() @@ -92,13 +89,6 @@ public void setUp() .setSystemProperty(FIELD_NAMES_IN_JSON_CAST_ENABLED, "true") .build(), new FeaturesConfig()); - - FeaturesConfig featuresConfigWithLegacyJsonCastEnabled = new FeaturesConfig() - .setLegacyJsonCast(true); - legacyJsonCastEnabled = new FunctionAssertions(session, featuresConfigWithLegacyJsonCastEnabled, true); - FeaturesConfig featuresConfigWithLegacyJsonCastDisabled = new FeaturesConfig() - .setLegacyJsonCast(false); - legacyJsonCastDisabled = new FunctionAssertions(session, featuresConfigWithLegacyJsonCastDisabled, true); } @AfterClass(alwaysRun = true) @@ -108,10 +98,6 @@ public final void tearDown() legacyRowFieldOrdinalAccess = null; fieldNameInJsonCastEnabled.close(); fieldNameInJsonCastEnabled = null; - legacyJsonCastEnabled.close(); - legacyJsonCastEnabled = null; - legacyJsonCastDisabled.close(); - legacyJsonCastDisabled = null; } @ScalarFunction @@ -418,8 +404,7 @@ public void testJsonToRow() asList(1L, 2L, null, 3L), null, asList(1L, 2L, 3L, null), null)); - // legacy json cast would not reserve the case of field name in json when casting to row type - legacyJsonCastEnabled.assertFunction("CAST(JSON '{" + + assertFunction("CAST(JSON '{" + "\"array2\": [1, 2, null, 3], " + "\"array1\": [], " + "\"array3\": null, " + @@ -455,89 +440,7 @@ public void testJsonToRow() asList(1L, 2L, null, 3L), null, null, asList(1L, 2L, 3L, null))); - // without legacy json cast, we would reserve the case of field name in json when casting to row type - legacyJsonCastDisabled.assertFunction("CAST(JSON '{" + - "\"array2\": [1, 2, null, 3], " + - "\"array1\": [], " + - "\"array3\": null, " + - "\"map3\": {\"a\": 1, \"b\": 2, \"none\": null, \"Three\": 3}, " + - "\"map1\": {}, " + - "\"map2\": null, " + - "\"rowAsJsonArray1\": [1, 2, null, 3], " + - "\"rowAsJsonArray2\": null, " + - "\"rowAsJsonObject2\": {\"a\": 1, \"b\": 2, \"none\": null, \"Three\": 3}, " + - "\"rowAsJsonObject1\": null}' " + - "AS ROW(array1 array, array2 ARRAY, array3 ARRAY, " + - "map1 MAP, map2 map, map3 MAP, " + - "\"rowAsJsonArray1\" row(BIGINT, bigint, BIGINT, BIGINT), \"rowAsJsonArray2\" ROW(BIGINT)," + - "\"rowAsJsonObject1\" ROW(nothing BIGINT), \"rowAsJsonObject2\" ROW(a BIGINT, b BIGINT, \"Three\" BIGINT, none BIGINT)))", - RowType.from(ImmutableList.of( - RowType.field("array1", new ArrayType(BIGINT)), - RowType.field("array2", new ArrayType(BIGINT)), - RowType.field("array3", new ArrayType(BIGINT)), - RowType.field("map1", mapType(VARCHAR, BIGINT)), - RowType.field("map2", mapType(VARCHAR, BIGINT)), - RowType.field("map3", mapType(VARCHAR, BIGINT)), - RowType.field("rowAsJsonArray1", RowType.anonymous(ImmutableList.of(BIGINT, BIGINT, BIGINT, BIGINT)), true), - RowType.field("rowAsJsonArray2", RowType.anonymous(ImmutableList.of(BIGINT)), true), - RowType.field("rowAsJsonObject1", RowType.from(ImmutableList.of(RowType.field("nothing", BIGINT))), true), - RowType.field("rowAsJsonObject2", RowType.from(ImmutableList.of( - RowType.field("a", BIGINT), - RowType.field("b", BIGINT), - RowType.field("Three", BIGINT, true), - RowType.field("none", BIGINT))), true))), - asList( - emptyList(), asList(1L, 2L, null, 3L), null, - ImmutableMap.of(), null, asMap(ImmutableList.of("a", "b", "none", "Three"), asList(1L, 2L, null, 3L)), - asList(1L, 2L, null, 3L), null, - null, asList(1L, 2L, 3L, null))); - - // legacy json cast would not reserve the case of field name in json when casting to row type - legacyJsonCastEnabled.assertFunction("CAST(json_extract('{\"1\":[{\"name\": \"John\", \"AGE\": 30}]}', '$') AS MAP>)", - mapType(BIGINT, new ArrayType(RowType.from(ImmutableList.of( - RowType.field("name", VARCHAR), - RowType.field("age", VARCHAR))))), - asMap(ImmutableList.of(1L), asList(asList(asList("John", "30"))))); - - // without legacy json cast, we would reserve the case of field name in json when casting to row type - legacyJsonCastDisabled.assertFunction("CAST(json_extract('{\"1\":[{\"name\": \"John\", \"AGE\": 30}]}', '$') AS MAP>)", - mapType(BIGINT, new ArrayType(RowType.from(ImmutableList.of( - RowType.field("name", VARCHAR), - RowType.field("AGE", VARCHAR, true))))), - asMap(ImmutableList.of(1L), asList(asList(asList("John", "30"))))); - - legacyJsonCastDisabled.assertFunction("CAST(json_extract('{\"1\":[{\"name\": \"John\", \"AGE\": 30}]}', '$') AS MAP>)", - mapType(BIGINT, new ArrayType(RowType.from(ImmutableList.of( - RowType.field("name", VARCHAR), - RowType.field("age", VARCHAR, true))))), - asMap(ImmutableList.of(1L), asList(asList(asList("John", null))))); - - legacyJsonCastDisabled.assertFunction("CAST(json_extract('{\"1\":[{\"name\": \"John\", \"AGE\": 30}]}', '$') AS MAP>)", - mapType(BIGINT, new ArrayType(RowType.from(ImmutableList.of( - RowType.field("name", VARCHAR), - RowType.field("age", VARCHAR))))), - asMap(ImmutableList.of(1L), asList(asList(asList("John", "30"))))); - - legacyJsonCastDisabled.assertFunction("CAST(json_extract('{\"1\":[{\"key1\": \"John\", \"KEY1\": \"Johnny\"}]}', '$') AS MAP>)", - mapType(BIGINT, new ArrayType(RowType.from(ImmutableList.of( - RowType.field("key1", VARCHAR, true), - RowType.field("KEY1", VARCHAR, true))))), - asMap(ImmutableList.of(1L), asList(asList(asList("John", "Johnny"))))); - - legacyJsonCastDisabled.assertFunction("CAST(json_extract('{\"1\":[{\"key1\": \"John\", \"KEY1\": \"Johnny\"}]}', '$') AS MAP>)", - mapType(BIGINT, new ArrayType(RowType.from(ImmutableList.of( - RowType.field("key1", VARCHAR), - RowType.field("KEY1", VARCHAR, true))))), - asMap(ImmutableList.of(1L), asList(asList(asList("John", "Johnny"))))); - - // invalid type definition - assertInvalidTypeDefinition("CAST(json_extract('{\"1\":[{\"key1\": \"John\", \"KEY1\": \"Johnny\"}]}', '$') AS MAP>)", - "Duplicate field: key1"); - // invalid cast - assertInvalidCast("CAST(json_extract('{\"1\":[{\"key1\": \"John\", \"KEY1\":\"Johnny\"}]}', '$') AS MAP>)", - "Cannot cast to map(bigint,array(row(key1 varchar))). Duplicate field: KEY1\n" + - "{\"1\":[{\"key1\":\"John\",\"KEY1\":\"Johnny\"}]}"); assertInvalidCast("CAST(unchecked_to_json('{\"a\":1,\"b\":2,\"a\":3}') AS ROW(a BIGINT, b BIGINT))", "Cannot cast to row(a bigint,b bigint). Duplicate field: a\n{\"a\":1,\"b\":2,\"a\":3}"); assertInvalidCast("CAST(unchecked_to_json('[{\"a\":1,\"b\":2,\"a\":3}]') AS ARRAY)", "Cannot cast to array(row(a bigint,b bigint)). Duplicate field: a\n[{\"a\":1,\"b\":2,\"a\":3}]"); } @@ -600,11 +503,7 @@ public void testRowCast() assertFunction("cast(row(1,null,3) as row(aa bigint, bb boolean, cc boolean)).aa", BIGINT, 1L); assertFunction("cast(row(null,null,null) as row(aa bigint, bb boolean, cc boolean)).aa", BIGINT, null); - // invalid type definition - assertInvalidTypeDefinition("CAST(ROW(1, 2) AS ROW(a BIGINT, A DOUBLE)).a", "Duplicate field: a"); - assertInvalidTypeDefinition("CAST(ROW(1, 2) AS ROW(KEY1 VARCHAR, \"key1\" VARCHAR))", "Duplicate field: key1"); - assertInvalidTypeDefinition("TYPEOF(CAST(row(1, 2) AS ROW(KEY1 VARCHAR, \"key1\" VARCHAR)))", "Duplicate field: key1"); - assertInvalidTypeDefinition("CAST(ROW(1, 2) AS ROW(KEY1 VARCHAR, \"key1\" VARCHAR)).key1", "Duplicate field: key1"); + assertInvalidFunction("CAST(ROW(1, 2) AS ROW(a BIGINT, A DOUBLE)).a"); // there are totally 7 field names String longFieldNameCast = "CAST(row(1.2E0, ARRAY[row(233, 6.9E0)], row(1000, 6.3E0)) AS ROW(%s VARCHAR, %s ARRAY(ROW(%s VARCHAR, %s VARCHAR)), %s ROW(%s VARCHAR, %s VARCHAR))).%s[1].%s"; @@ -626,14 +525,6 @@ public void testRowCast() RowType.field("d", VARCHAR), RowType.field("e", new ArrayType(BIGINT)))), asList(2L, 1.5, true, "abc", ImmutableList.of(1L, 2L))); - - assertFunction( - "MAP(ARRAY['myFirstRow', 'mySecondRow'], ARRAY[cast(row('row1FieldValue1', 'row1FieldValue2') as row(\"firstField\" varchar, \"secondField\" varchar)), cast(row('row2FieldValue1', 'row2FieldValue2') as row(\"firstField\" varchar, \"secondField\" varchar))])", - mapType(VarcharType.createVarcharType(11), RowType.from(ImmutableList.of( - RowType.field("firstField", VARCHAR, true), - RowType.field("secondField", VARCHAR, true)))), - ImmutableMap.of("myFirstRow", asList("row1FieldValue1", "row1FieldValue2"), - "mySecondRow", asList("row2FieldValue1", "row2FieldValue2"))); } @Test diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Cast.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Cast.java index 312474f4840e..cc9bd303be27 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Cast.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Cast.java @@ -19,6 +19,7 @@ import java.util.Objects; import java.util.Optional; +import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; public final class Cast @@ -66,7 +67,7 @@ public Cast(Optional location, Expression expression, String type, requireNonNull(type, "type is null"); this.expression = expression; - this.type = transformCase(type); + this.type = type.toLowerCase(ENGLISH); this.safe = safe; this.typeOnly = typeOnly; } @@ -91,34 +92,6 @@ public boolean isTypeOnly() return typeOnly; } - /** - * We uniformly change the whole type string to lower case, - * except field names which are enclosed in double quotation marks - * */ - private static String transformCase(String input) - { - if (input == null) { - return null; - } - - StringBuilder sb = new StringBuilder(); - boolean insideQuotes = false; - for (int i = 0, l = input.length(); i < l; i++) { - char ch = input.charAt(i); - if (ch == '"') { - insideQuotes = !insideQuotes; - } - - if (insideQuotes) { - sb.append(ch); - } - else { - sb.append(Character.toLowerCase(ch)); - } - } - return sb.toString(); - } - @Override public R accept(AstVisitor visitor, C context) { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/StandardErrorCode.java b/presto-spi/src/main/java/com/facebook/presto/spi/StandardErrorCode.java index 17169181ca3d..2aef128032b8 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/StandardErrorCode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/StandardErrorCode.java @@ -70,7 +70,6 @@ public enum StandardErrorCode WARNING_AS_ERROR(0x0000_002C, USER_ERROR), INVALID_ARGUMENTS(0x0000_002D, USER_ERROR), EXCEEDED_PLAN_NODE_LIMIT(0x0000_002E, USER_ERROR), - INVALID_TYPE_DEFINITION(0x0000_002F, USER_ERROR), GENERIC_INTERNAL_ERROR(0x0001_0000, INTERNAL_ERROR), TOO_MANY_REQUESTS_FAILED(0x0001_0001, INTERNAL_ERROR, true),