Skip to content

Commit

Permalink
Add a new ErrorCode UNKNOWN_TYPE
Browse files Browse the repository at this point in the history
  • Loading branch information
pdabre12 authored and Pratik Joseph Dabre committed Feb 5, 2025
1 parent a7a7f80 commit 79821f4
Show file tree
Hide file tree
Showing 8 changed files with 61 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.facebook.presto.spi.ColumnMetadata;
import com.facebook.presto.spi.ConnectorId;
import com.facebook.presto.spi.MaterializedViewDefinition;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.TableHandle;
import com.facebook.presto.spi.WarningCollector;
import com.facebook.presto.spi.security.AccessControl;
Expand All @@ -39,6 +40,7 @@
import static com.facebook.presto.common.type.UnknownType.UNKNOWN;
import static com.facebook.presto.metadata.MetadataUtil.createQualifiedObjectName;
import static com.facebook.presto.metadata.MetadataUtil.getConnectorIdOrThrow;
import static com.facebook.presto.spi.StandardErrorCode.UNKNOWN_TYPE;
import static com.facebook.presto.spi.connector.ConnectorCapabilities.NOT_NULL_COLUMN_CONSTRAINT;
import static com.facebook.presto.sql.NodeUtils.mapFromProperties;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.COLUMN_ALREADY_EXISTS;
Expand Down Expand Up @@ -88,8 +90,11 @@ public ListenableFuture<?> execute(AddColumn statement, TransactionManager trans
try {
type = metadata.getType(parseTypeSignature(element.getType()));
}
catch (IllegalArgumentException e) {
throw new SemanticException(TYPE_MISMATCH, element, "Unknown type '%s' for column '%s'", element.getType(), element.getName());
catch (PrestoException e) {
if (e.getErrorCode() == UNKNOWN_TYPE.toErrorCode()) {
throw new SemanticException(TYPE_MISMATCH, element, "Unknown type '%s' for column '%s'", element.getType(), element.getName());
}
throw e;
}
if (type.equals(UNKNOWN)) {
throw new SemanticException(TYPE_MISMATCH, element, "Unknown type '%s' for column '%s'", element.getType(), element.getName());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@
import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS;
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static com.facebook.presto.spi.StandardErrorCode.SYNTAX_ERROR;
import static com.facebook.presto.spi.StandardErrorCode.UNKNOWN_TYPE;
import static com.facebook.presto.spi.connector.ConnectorCapabilities.NOT_NULL_COLUMN_CONSTRAINT;
import static com.facebook.presto.sql.NodeUtils.mapFromProperties;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.DUPLICATE_COLUMN_NAME;
Expand Down Expand Up @@ -125,8 +126,11 @@ public ListenableFuture<?> internalExecute(CreateTable statement, Metadata metad
try {
type = metadata.getType(parseTypeSignature(column.getType()));
}
catch (IllegalArgumentException e) {
throw new SemanticException(TYPE_MISMATCH, element, "Unknown type '%s' for column '%s'", column.getType(), column.getName());
catch (PrestoException e) {
if (e.getErrorCode() == UNKNOWN_TYPE.toErrorCode()) {
throw new SemanticException(TYPE_MISMATCH, element, "Unknown type '%s' for column '%s'", column.getType(), column.getName());
}
throw e;
}
if (type.equals(UNKNOWN)) {
throw new SemanticException(TYPE_MISMATCH, element, "Unknown type '%s' for column '%s'", column.getType(), column.getName());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -453,6 +453,7 @@
import static com.facebook.presto.operator.window.AggregateWindowFunction.supplier;
import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_MISSING;
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_USER_ERROR;
import static com.facebook.presto.spi.StandardErrorCode.UNKNOWN_TYPE;
import static com.facebook.presto.spi.function.FunctionImplementationType.JAVA;
import static com.facebook.presto.spi.function.FunctionImplementationType.SQL;
import static com.facebook.presto.spi.function.FunctionKind.AGGREGATE;
Expand Down Expand Up @@ -1312,7 +1313,7 @@ private Type instantiateParametricType(ExactTypeSignature exactSignature)

ParametricType parametricType = parametricTypes.get(signature.getBase().toLowerCase(Locale.ENGLISH));
if (parametricType == null) {
throw new IllegalArgumentException("Unknown type " + signature);
throw new PrestoException(UNKNOWN_TYPE, "Unknown type " + signature);
}

if (parametricType instanceof MapParametricType) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.facebook.presto.common.type.TypeSignature;
import com.facebook.presto.common.type.TypeSignatureParameter;
import com.facebook.presto.common.type.TypeWithName;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.function.LongVariableConstraint;
import com.facebook.presto.spi.function.Signature;
import com.facebook.presto.spi.function.TypeVariableConstraint;
Expand All @@ -39,6 +40,7 @@
import java.util.Set;

import static com.facebook.presto.common.type.UnknownType.UNKNOWN;
import static com.facebook.presto.spi.StandardErrorCode.UNKNOWN_TYPE;
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes;
import static com.facebook.presto.type.TypeCalculation.calculateLiteralValue;
import static com.google.common.base.Preconditions.checkArgument;
Expand Down Expand Up @@ -532,7 +534,15 @@ private static List<TypeSignature> expandVarargFormalTypeSignature(List<TypeSign
private boolean satisfiesCoercion(boolean allowCoercion, Type fromType, TypeSignature toTypeSignature)
{
if (allowCoercion) {
return functionAndTypeManager.canCoerce(fromType, functionAndTypeManager.getType(toTypeSignature));
try {
return functionAndTypeManager.canCoerce(fromType, functionAndTypeManager.getType(toTypeSignature));
}
catch (PrestoException e) {
if (e.getErrorCode() == UNKNOWN_TYPE.toErrorCode()) {
return false;
}
throw e;
}
}
else if (fromType.getTypeSignature().equals(toTypeSignature)) {
return true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,7 @@
import static com.facebook.presto.common.type.VarcharType.VARCHAR;
import static com.facebook.presto.metadata.BuiltInTypeAndFunctionNamespaceManager.JAVA_BUILTIN_NAMESPACE;
import static com.facebook.presto.spi.StandardErrorCode.OPERATOR_NOT_FOUND;
import static com.facebook.presto.spi.StandardErrorCode.UNKNOWN_TYPE;
import static com.facebook.presto.spi.StandardWarningCode.SEMANTIC_WARNING;
import static com.facebook.presto.sql.NodeUtils.getSortItemsFromOrderBy;
import static com.facebook.presto.sql.analyzer.Analyzer.verifyNoAggregateWindowOrGroupingFunctions;
Expand Down Expand Up @@ -889,8 +890,11 @@ protected Type visitGenericLiteral(GenericLiteral node, StackableAstVisitorConte
try {
type = functionAndTypeResolver.getType(parseTypeSignature(node.getType()));
}
catch (IllegalArgumentException e) {
throw new SemanticException(TYPE_MISMATCH, node, "Unknown type: " + node.getType());
catch (PrestoException e) {
if (e.getErrorCode() == UNKNOWN_TYPE.toErrorCode()) {
throw new SemanticException(TYPE_MISMATCH, node, "Unknown type: " + node.getType());
}
throw e;
}

if (!JSON.equals(type)) {
Expand All @@ -912,8 +916,11 @@ protected Type visitEnumLiteral(EnumLiteral node, StackableAstVisitorContext<Con
try {
type = functionAndTypeResolver.getType(parseTypeSignature(node.getType()));
}
catch (IllegalArgumentException e) {
throw new SemanticException(TYPE_MISMATCH, node, "Unknown type: " + node.getType());
catch (PrestoException e) {
if (e.getErrorCode() == UNKNOWN_TYPE.toErrorCode()) {
throw new SemanticException(TYPE_MISMATCH, node, "Unknown type: " + node.getType());
}
throw e;
}

return setExpressionType(node, type);
Expand Down Expand Up @@ -1342,6 +1349,12 @@ public Type visitCast(Cast node, StackableAstVisitorContext<Context> context)
catch (IllegalArgumentException e) {
throw new SemanticException(TYPE_MISMATCH, node, "Unknown type: " + node.getType());
}
catch (PrestoException e) {
if (e.getErrorCode() == UNKNOWN_TYPE.toErrorCode()) {
throw new SemanticException(TYPE_MISMATCH, node, "Unknown type: " + node.getType());
}
throw e;
}

if (type.equals(UNKNOWN)) {
throw new SemanticException(TYPE_MISMATCH, node, "UNKNOWN is not a valid type");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import com.facebook.presto.common.type.EnumType;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeWithName;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.SourceLocation;
import com.facebook.presto.spi.function.FunctionHandle;
import com.facebook.presto.spi.relation.VariableReferenceExpression;
Expand Down Expand Up @@ -43,6 +44,7 @@
import java.util.function.Predicate;

import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.spi.StandardErrorCode.UNKNOWN_TYPE;
import static com.facebook.presto.spi.function.FunctionKind.AGGREGATE;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
Expand Down Expand Up @@ -152,8 +154,11 @@ public static Optional<TypeWithName> tryResolveEnumLiteralType(QualifiedName qua
return Optional.of((TypeWithName) baseType);
}
}
catch (IllegalArgumentException e) {
return Optional.empty();
catch (PrestoException e) {
if (e.getErrorCode() == UNKNOWN_TYPE.toErrorCode()) {
return Optional.empty();
}
throw e;
}
return Optional.empty();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.facebook.presto.common.type.TypeSignature;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.OperatorNotFoundException;
import com.facebook.presto.spi.PrestoException;
import com.google.common.collect.ImmutableSet;
import org.testng.annotations.Test;

Expand Down Expand Up @@ -52,6 +53,7 @@
import static com.facebook.presto.common.type.VarcharType.createVarcharType;
import static com.facebook.presto.metadata.CastType.CAST;
import static com.facebook.presto.metadata.FunctionAndTypeManager.createTestFunctionAndTypeManager;
import static com.facebook.presto.spi.StandardErrorCode.UNKNOWN_TYPE;
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes;
import static com.facebook.presto.type.JoniRegexpType.JONI_REGEXP;
import static com.facebook.presto.type.JsonPathType.JSON_PATH;
Expand All @@ -73,13 +75,17 @@ public void testNonexistentType()
{
try {
functionAndTypeManager.getType(parseTypeSignature("not a real type"));
fail("Expect to throw IllegalArgumentException");
fail("Expect to throw PrestoException");
}
catch (IllegalArgumentException e) {
assertTrue(e.getMessage().matches("Unknown type.*"));
catch (PrestoException e) {
if (e.getErrorCode() == UNKNOWN_TYPE.toErrorCode()) {
assertTrue(e.getMessage().matches("Unknown type.*"));
return;
}
throw e;
}
catch (Throwable t) {
fail("Expect to throw IllegalArgumentException, got " + t.getClass());
fail("Expect to throw PrestoException, got " + t.getClass());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ public enum StandardErrorCode
INVALID_TYPE_DEFINITION(0x0000_002F, USER_ERROR),
VIEW_NOT_FOUND(0x0000_0030, USER_ERROR),
INVALID_LIMIT_CLAUSE(0x0000_0031, USER_ERROR),
UNKNOWN_TYPE(0x0000_0032, USER_ERROR),

GENERIC_INTERNAL_ERROR(0x0001_0000, INTERNAL_ERROR),
TOO_MANY_REQUESTS_FAILED(0x0001_0001, INTERNAL_ERROR, true),
Expand Down

0 comments on commit 79821f4

Please sign in to comment.