Skip to content

Commit

Permalink
Report error position when evaluating properties
Browse files Browse the repository at this point in the history
  • Loading branch information
electrum committed May 9, 2024
1 parent ec64be7 commit 6a0799d
Show file tree
Hide file tree
Showing 16 changed files with 74 additions and 75 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
import io.trino.Session;
import io.trino.execution.warnings.WarningCollector;
import io.trino.metadata.CatalogManager;
import io.trino.metadata.PropertyUtil;
import io.trino.security.AccessControl;
import io.trino.spi.TrinoException;
import io.trino.spi.catalog.CatalogName;
Expand All @@ -34,9 +33,11 @@

import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
import static io.trino.execution.ParameterExtractor.bindParameters;
import static io.trino.metadata.PropertyUtil.evaluateProperty;
import static io.trino.spi.StandardErrorCode.INVALID_CATALOG_PROPERTY;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static io.trino.sql.analyzer.ExpressionTreeUtils.extractLocation;
import static io.trino.sql.analyzer.SemanticExceptions.semanticException;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
Expand Down Expand Up @@ -84,7 +85,8 @@ public ListenableFuture<Void> execute(
}
properties.put(
property.getName().getValue(),
(String) PropertyUtil.evaluateProperty(
(String) evaluateProperty(
extractLocation(property),
property.getName().getValue(),
VARCHAR,
property.getNonDefaultValue(),
Expand Down
42 changes: 15 additions & 27 deletions core/trino-main/src/main/java/io/trino/metadata/PropertyUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import io.trino.Session;
import io.trino.security.AccessControl;
import io.trino.spi.ErrorCodeSupplier;
import io.trino.spi.Location;
import io.trino.spi.TrinoException;
import io.trino.spi.block.Block;
import io.trino.spi.session.PropertyMetadata;
Expand All @@ -46,8 +47,8 @@
import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.spi.type.TypeUtils.writeNativeValue;
import static io.trino.sql.analyzer.ConstantEvaluator.evaluateConstant;
import static io.trino.sql.analyzer.ExpressionTreeUtils.extractLocation;
import static io.trino.util.MoreLists.mappedCopy;
import static java.lang.String.format;
import static java.util.Locale.ENGLISH;

public final class PropertyUtil
Expand All @@ -69,11 +70,13 @@ public static Map<String, Optional<Object>> evaluateProperties(

// Fill in user-specified properties
for (Property property : setProperties) {
Optional<Location> location = extractLocation(property);
// property names are case-insensitive and normalized to lower case
String propertyName = property.getName().getValue().toLowerCase(ENGLISH);
PropertyMetadata<?> propertyMetadata = metadata.get(propertyName);
if (propertyMetadata == null) {
throw new TrinoException(errorCode, format("%s '%s' does not exist", capitalize(propertyTypeDescription), propertyName));
String message = "%s '%s' does not exist".formatted(capitalize(propertyTypeDescription), propertyName);
throw new TrinoException(errorCode, location, message, null);
}

Optional<Object> value;
Expand All @@ -82,6 +85,7 @@ public static Map<String, Optional<Object>> evaluateProperties(
}
else {
value = Optional.of(evaluateProperty(
location,
property.getNonDefaultValue(),
propertyMetadata,
session,
Expand All @@ -106,6 +110,7 @@ public static Map<String, Optional<Object>> evaluateProperties(
}

private static Object evaluateProperty(
Optional<Location> location,
Expression expression,
PropertyMetadata<?> property,
Session session,
Expand All @@ -116,6 +121,7 @@ private static Object evaluateProperty(
String propertyTypeDescription)
{
Object sqlObjectValue = evaluateProperty(
location,
property.getName(),
property.getSqlType(),
expression,
Expand All @@ -130,19 +136,13 @@ private static Object evaluateProperty(
return property.decode(sqlObjectValue);
}
catch (Exception e) {
throw new TrinoException(
errorCode,
format(
"Unable to set %s '%s' to [%s]: %s",
propertyTypeDescription,
property.getName(),
expression,
e.getMessage()),
e);
String message = "Unable to set %s '%s' to [%s]: %s".formatted(propertyTypeDescription, property.getName(), expression, e.getMessage());
throw new TrinoException(errorCode, location, message, e);
}
}

public static Object evaluateProperty(
Optional<Location> location,
String propertyName,
Type propertyType,
Expression expression,
Expand All @@ -163,25 +163,13 @@ public static Object evaluateProperty(
sqlObjectValue = propertyType.getObjectValue(session.toConnectorSession(), block, 0);
}
catch (TrinoException e) {
throw new TrinoException(
errorCode,
format(
"Invalid value for %s '%s': Cannot convert [%s] to %s",
propertyTypeDescription,
propertyName,
expression,
propertyType),
e);
String message = "Invalid value for %s '%s': Cannot convert [%s] to %s".formatted(propertyTypeDescription, propertyName, expression, propertyType);
throw new TrinoException(errorCode, location, message, e);
}

if (sqlObjectValue == null) {
throw new TrinoException(
errorCode,
format(
"Invalid null value for %s '%s' from [%s]",
propertyTypeDescription,
propertyName,
expression));
String message = "Invalid null value for %s '%s' from [%s]".formatted(propertyTypeDescription, propertyName, expression);
throw new TrinoException(errorCode, location, message, null);
}
return sqlObjectValue;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,16 +22,15 @@
import io.trino.spi.session.PropertyMetadata;
import io.trino.sql.PlannerContext;
import io.trino.sql.tree.Expression;
import io.trino.sql.tree.Identifier;
import io.trino.sql.tree.NodeRef;
import io.trino.sql.tree.Parameter;
import io.trino.sql.tree.Property;

import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static io.trino.metadata.PropertyUtil.evaluateProperties;
import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT;
Expand All @@ -51,7 +50,7 @@ public Map<String, Object> getProperties(
String catalogName,
CatalogHandle catalogHandle,
String procedureName,
Map<String, Expression> sqlPropertyValues,
List<Property> properties,
Session session,
PlannerContext plannerContext,
AccessControl accessControl,
Expand All @@ -61,9 +60,7 @@ public Map<String, Object> getProperties(
Map<String, PropertyMetadata<?>> supportedProperties = Maps.uniqueIndex(tableProcedure.getProperties(), PropertyMetadata::getName);

Map<String, Optional<Object>> propertyValues = evaluateProperties(
sqlPropertyValues.entrySet().stream()
.map(entry -> new Property(new Identifier(entry.getKey()), entry.getValue()))
.collect(toImmutableList()),
properties,
session,
plannerContext,
accessControl,
Expand All @@ -72,6 +69,7 @@ public Map<String, Object> getProperties(
supportedProperties,
INVALID_PROCEDURE_ARGUMENT,
format("catalog '%s' table procedure '%s' property", catalogName, procedureName));

return propertyValues.entrySet().stream()
.filter(entry -> entry.getValue().isPresent())
.collect(toImmutableMap(Entry::getKey, entry -> entry.getValue().orElseThrow()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1276,12 +1276,12 @@ protected Scope visitTableExecute(TableExecute node, Optional<Scope> scope)

// analyze arguments

Map<String, Expression> propertiesMap = processTableExecuteArguments(node, procedureMetadata, scope);
List<Property> arguments = processTableExecuteArguments(node, procedureMetadata, scope);
Map<String, Object> tableProperties = tableProceduresPropertyManager.getProperties(
catalogName,
catalogHandle,
procedureName,
propertiesMap,
arguments,
session,
plannerContext,
accessControl,
Expand All @@ -1304,7 +1304,7 @@ protected Scope visitTableExecute(TableExecute node, Optional<Scope> scope)
return createAndAssignScope(node, scope, Field.newUnqualified("rows", BIGINT));
}

private Map<String, Expression> processTableExecuteArguments(TableExecute node, TableProcedureMetadata procedureMetadata, Optional<Scope> scope)
private List<Property> processTableExecuteArguments(TableExecute node, TableProcedureMetadata procedureMetadata, Optional<Scope> scope)
{
List<CallArgument> arguments = node.getArguments();
Predicate<CallArgument> hasName = argument -> argument.getName().isPresent();
Expand All @@ -1322,25 +1322,29 @@ private Map<String, Expression> processTableExecuteArguments(TableExecute node,
process(argument, scope);
}

Map<String, Expression> argumentsMap = new HashMap<>();
List<Property> properties = new ArrayList<>();

if (anyNamed) {
// all properties named
Set<String> names = new HashSet<>();
for (CallArgument argument : arguments) {
if (argumentsMap.put(argument.getName().get().getCanonicalValue(), argument.getValue()) != null) {
throw semanticException(DUPLICATE_PROPERTY, argument, "Duplicate named argument: %s", argument.getName());
Identifier name = argument.getName().orElseThrow();
if (!names.add(name.getCanonicalValue())) {
throw semanticException(DUPLICATE_PROPERTY, argument, "Duplicate named argument: %s", name);
}
properties.add(new Property(argument.getLocation(), name, argument.getValue()));
}
}
else {
// all properties unnamed
int pos = 0;
for (CallArgument argument : arguments) {
argumentsMap.put(procedureMetadata.getProperties().get(pos).getName(), argument.getValue());
Identifier name = new Identifier(procedureMetadata.getProperties().get(pos).getName());
properties.add(new Property(argument.getLocation(), name, argument.getValue()));
pos++;
}
}
return ImmutableMap.copyOf(argumentsMap);
return properties;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import io.trino.sql.tree.AllColumns;
import io.trino.sql.tree.CreateMaterializedView;
import io.trino.sql.tree.Identifier;
import io.trino.sql.tree.NodeLocation;
import io.trino.sql.tree.Property;
import io.trino.sql.tree.QualifiedName;
import io.trino.sql.tree.Statement;
Expand Down Expand Up @@ -169,13 +170,13 @@ void testCreateMaterializedViewWithInvalidProperty()
false,
true,
Optional.empty(),
ImmutableList.of(new Property(new Identifier("baz"), new StringLiteral("abc"))),
ImmutableList.of(new Property(new NodeLocation(1, 88), new Identifier("baz"), new StringLiteral("abc"))),
Optional.empty());

queryRunner.inTransaction(transactionSession -> {
assertTrinoExceptionThrownBy(() -> createMaterializedView(transactionSession, statement))
.hasErrorCode(INVALID_MATERIALIZED_VIEW_PROPERTY)
.hasMessage("Catalog 'test_catalog' materialized view property 'baz' does not exist");
.hasMessage("line 1:88: Catalog 'test_catalog' materialized view property 'baz' does not exist");
assertThat(metadata.getCreateMaterializedViewCallCount()).isEqualTo(0);
return null;
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import io.trino.sql.tree.AllColumns;
import io.trino.sql.tree.CreateView;
import io.trino.sql.tree.Identifier;
import io.trino.sql.tree.NodeLocation;
import io.trino.sql.tree.Property;
import io.trino.sql.tree.QualifiedName;
import io.trino.sql.tree.Query;
Expand Down Expand Up @@ -149,10 +150,10 @@ public void testCreateViewWithUnknownProperty()

assertTrinoExceptionThrownBy(() -> getFutureValue(executeCreateView(
asQualifiedName(viewName),
ImmutableList.of(new Property(new Identifier("unknown_property"), new StringLiteral("unknown"))),
ImmutableList.of(new Property(new NodeLocation(1, 88), new Identifier("unknown_property"), new StringLiteral("unknown"))),
false)))
.hasErrorCode(INVALID_VIEW_PROPERTY)
.hasMessage("Catalog 'test_catalog' view property 'unknown_property' does not exist");
.hasMessage("line 1:88: Catalog 'test_catalog' view property 'unknown_property' does not exist");
}

@Test
Expand All @@ -162,10 +163,10 @@ public void testCreateViewWithInvalidProperty()

assertTrinoExceptionThrownBy(() -> getFutureValue(executeCreateView(
asQualifiedName(viewName),
ImmutableList.of(new Property(new Identifier("boolean_property"), new StringLiteral("unknown"))),
ImmutableList.of(new Property(new NodeLocation(1, 88), new Identifier("boolean_property"), new StringLiteral("unknown"))),
false)))
.hasErrorCode(INVALID_VIEW_PROPERTY)
.hasMessage("Invalid value for catalog 'test_catalog' view property 'boolean_property': Cannot convert ['unknown'] to boolean");
.hasMessage("line 1:88: Invalid value for catalog 'test_catalog' view property 'boolean_property': Cannot convert ['unknown'] to boolean");
}

private ListenableFuture<Void> executeCreateView(QualifiedName viewName, boolean replace)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,11 @@ public Property(NodeLocation location, Identifier name, Expression value)
Optional.of(requireNonNull(value, "value is null")));
}

public Property(Optional<NodeLocation> location, Identifier name, Expression value)
{
this(location, name, Optional.of(requireNonNull(value, "value is null")));
}

private Property(Optional<NodeLocation> location, Identifier name, Optional<Expression> value)
{
super(location);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -369,7 +369,7 @@ public void testDifferentEngine()

//NOT support engine
assertQueryFails("CREATE TABLE " + tableName + " (id int NOT NULL, x VARCHAR) WITH (engine = 'bad_engine')",
"Unable to set catalog 'clickhouse' table property 'engine' to.*");
"line 1:76: Unable to set catalog 'clickhouse' table property 'engine' to.*");
}

@Test
Expand Down Expand Up @@ -552,7 +552,7 @@ public void testAlterInvalidTableProperties()
"(p1 int NOT NULL, p2 int NOT NULL, x VARCHAR) WITH (engine = 'MergeTree', order_by = ARRAY['p1', 'p2'], primary_key = ARRAY['p1', 'p2'])")) {
assertQueryFails(
"ALTER TABLE " + table.getName() + " SET PROPERTIES invalid_property = 'p2'",
"Catalog 'clickhouse' table property 'invalid_property' does not exist");
"line 1:66: Catalog 'clickhouse' table property 'invalid_property' does not exist");
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1856,10 +1856,10 @@ public void testOptimizeParameterValidation()
format("line 1:7: Table 'delta.%s.no_such_table_exists' does not exist", SCHEMA));
assertQueryFails(
"ALTER TABLE nation EXECUTE OPTIMIZE (file_size_threshold => '33')",
"\\QUnable to set catalog 'delta' table procedure 'OPTIMIZE' property 'file_size_threshold' to ['33']: size is not a valid data size string: 33");
"\\Qline 1:38: Unable to set catalog 'delta' table procedure 'OPTIMIZE' property 'file_size_threshold' to ['33']: size is not a valid data size string: 33");
assertQueryFails(
"ALTER TABLE nation EXECUTE OPTIMIZE (file_size_threshold => '33s')",
"\\QUnable to set catalog 'delta' table procedure 'OPTIMIZE' property 'file_size_threshold' to ['33s']: Unknown unit: s");
"\\Qline 1:38: Unable to set catalog 'delta' table procedure 'OPTIMIZE' property 'file_size_threshold' to ['33s']: Unknown unit: s");
}

@Test
Expand Down

0 comments on commit 6a0799d

Please sign in to comment.