diff --git a/core/trino-main/src/main/java/io/trino/execution/CreateCatalogTask.java b/core/trino-main/src/main/java/io/trino/execution/CreateCatalogTask.java index 453cf239ed6027..2a9ad0fc4b2711 100644 --- a/core/trino-main/src/main/java/io/trino/execution/CreateCatalogTask.java +++ b/core/trino-main/src/main/java/io/trino/execution/CreateCatalogTask.java @@ -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; @@ -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; @@ -84,7 +85,8 @@ public ListenableFuture execute( } properties.put( property.getName().getValue(), - (String) PropertyUtil.evaluateProperty( + (String) evaluateProperty( + extractLocation(property), property.getName().getValue(), VARCHAR, property.getNonDefaultValue(), diff --git a/core/trino-main/src/main/java/io/trino/metadata/PropertyUtil.java b/core/trino-main/src/main/java/io/trino/metadata/PropertyUtil.java index 546db8ec8c763e..7c3e5f41b5408a 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/PropertyUtil.java +++ b/core/trino-main/src/main/java/io/trino/metadata/PropertyUtil.java @@ -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; @@ -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 @@ -69,11 +70,13 @@ public static Map> evaluateProperties( // Fill in user-specified properties for (Property property : setProperties) { + Optional 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 value; @@ -82,6 +85,7 @@ public static Map> evaluateProperties( } else { value = Optional.of(evaluateProperty( + location, property.getNonDefaultValue(), propertyMetadata, session, @@ -106,6 +110,7 @@ public static Map> evaluateProperties( } private static Object evaluateProperty( + Optional location, Expression expression, PropertyMetadata property, Session session, @@ -116,6 +121,7 @@ private static Object evaluateProperty( String propertyTypeDescription) { Object sqlObjectValue = evaluateProperty( + location, property.getName(), property.getSqlType(), expression, @@ -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, String propertyName, Type propertyType, Expression expression, @@ -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; } diff --git a/core/trino-main/src/main/java/io/trino/metadata/TableProceduresPropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/TableProceduresPropertyManager.java index d191d2a18621ab..61a879f4f97322 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/TableProceduresPropertyManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/TableProceduresPropertyManager.java @@ -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; @@ -51,7 +50,7 @@ public Map getProperties( String catalogName, CatalogHandle catalogHandle, String procedureName, - Map sqlPropertyValues, + List properties, Session session, PlannerContext plannerContext, AccessControl accessControl, @@ -61,9 +60,7 @@ public Map getProperties( Map> supportedProperties = Maps.uniqueIndex(tableProcedure.getProperties(), PropertyMetadata::getName); Map> propertyValues = evaluateProperties( - sqlPropertyValues.entrySet().stream() - .map(entry -> new Property(new Identifier(entry.getKey()), entry.getValue())) - .collect(toImmutableList()), + properties, session, plannerContext, accessControl, @@ -72,6 +69,7 @@ public Map 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())); diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java index 4ba24ecda9b112..71684735e86b21 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java @@ -1276,12 +1276,12 @@ protected Scope visitTableExecute(TableExecute node, Optional scope) // analyze arguments - Map propertiesMap = processTableExecuteArguments(node, procedureMetadata, scope); + List arguments = processTableExecuteArguments(node, procedureMetadata, scope); Map tableProperties = tableProceduresPropertyManager.getProperties( catalogName, catalogHandle, procedureName, - propertiesMap, + arguments, session, plannerContext, accessControl, @@ -1304,7 +1304,7 @@ protected Scope visitTableExecute(TableExecute node, Optional scope) return createAndAssignScope(node, scope, Field.newUnqualified("rows", BIGINT)); } - private Map processTableExecuteArguments(TableExecute node, TableProcedureMetadata procedureMetadata, Optional scope) + private List processTableExecuteArguments(TableExecute node, TableProcedureMetadata procedureMetadata, Optional scope) { List arguments = node.getArguments(); Predicate hasName = argument -> argument.getName().isPresent(); @@ -1322,25 +1322,29 @@ private Map processTableExecuteArguments(TableExecute node, process(argument, scope); } - Map argumentsMap = new HashMap<>(); + List properties = new ArrayList<>(); if (anyNamed) { // all properties named + Set 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 diff --git a/core/trino-main/src/test/java/io/trino/execution/TestCreateMaterializedViewTask.java b/core/trino-main/src/test/java/io/trino/execution/TestCreateMaterializedViewTask.java index 800251754505b3..0fad1ec5dcee78 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestCreateMaterializedViewTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestCreateMaterializedViewTask.java @@ -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; @@ -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; }); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestCreateViewTask.java b/core/trino-main/src/test/java/io/trino/execution/TestCreateViewTask.java index c87943cfa774b8..a2f4578143d95b 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestCreateViewTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestCreateViewTask.java @@ -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; @@ -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 @@ -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 executeCreateView(QualifiedName viewName, boolean replace) diff --git a/core/trino-parser/src/main/java/io/trino/sql/tree/Property.java b/core/trino-parser/src/main/java/io/trino/sql/tree/Property.java index bde9df4f3ca33b..440fa142c40d7e 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/tree/Property.java +++ b/core/trino-parser/src/main/java/io/trino/sql/tree/Property.java @@ -59,6 +59,11 @@ public Property(NodeLocation location, Identifier name, Expression value) Optional.of(requireNonNull(value, "value is null"))); } + public Property(Optional location, Identifier name, Expression value) + { + this(location, name, Optional.of(requireNonNull(value, "value is null"))); + } + private Property(Optional location, Identifier name, Optional value) { super(location); diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 793b243cb07ee9..0aaf0a9a6b1033 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -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 @@ -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"); } } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java index 19e93546b0347b..352b38e67e1c44 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java @@ -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 diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index 0f11867cc84093..cdd47265ac7ab4 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -6746,9 +6746,9 @@ public void testInvalidAnalyzePartitionedTable() createPartitionedTableForAnalyzeTest(tableName); // Test invalid property - assertQueryFails(format("ANALYZE %s WITH (error = 1)", tableName), ".*'hive' analyze property 'error' does not exist.*"); - assertQueryFails(format("ANALYZE %s WITH (partitions = 1)", tableName), "\\QInvalid value for catalog 'hive' analyze property 'partitions': Cannot convert [1] to array(array(varchar))\\E"); - assertQueryFails(format("ANALYZE %s WITH (partitions = NULL)", tableName), "\\QInvalid null value for catalog 'hive' analyze property 'partitions' from [null]\\E"); + assertQueryFails(format("ANALYZE %s WITH (error = 1)", tableName), "line 1:64: Catalog 'hive' analyze property 'error' does not exist"); + assertQueryFails(format("ANALYZE %s WITH (partitions = 1)", tableName), "\\Qline 1:64: Invalid value for catalog 'hive' analyze property 'partitions': Cannot convert [1] to array(array(varchar))\\E"); + assertQueryFails(format("ANALYZE %s WITH (partitions = NULL)", tableName), "\\Qline 1:64: Invalid null value for catalog 'hive' analyze property 'partitions' from [null]\\E"); assertQueryFails(format("ANALYZE %s WITH (partitions = ARRAY[NULL])", tableName), ".*Invalid null value in analyze partitions property.*"); // Test non-existed partition @@ -7320,7 +7320,7 @@ public void testInvalidColumnsAnalyzeTable() // Column names must be strings assertQueryFails( "ANALYZE " + tableName + " WITH (columns = ARRAY[42])", - "\\QInvalid value for catalog 'hive' analyze property 'columns': Cannot convert [ARRAY[42]] to array(varchar)\\E"); + "\\Qline 1:52: Invalid value for catalog 'hive' analyze property 'columns': Cannot convert [ARRAY[42]] to array(varchar)\\E"); assertUpdate("DROP TABLE " + tableName); } @@ -9067,10 +9067,10 @@ public void testDuplicateExtraProperties() { assertQueryFails( "CREATE TABLE create_table_with_duplicate_extra_properties (c1 integer) WITH (extra_properties = MAP(ARRAY['extra.property', 'extra.property'], ARRAY['true', 'false']))", - "Invalid value for catalog 'hive' table property 'extra_properties': Cannot convert.*"); + "line 1:78: Invalid value for catalog 'hive' table property 'extra_properties': Cannot convert.*"); assertQueryFails( "CREATE TABLE create_table_select_as_with_duplicate_extra_properties (c1 integer) WITH (extra_properties = MAP(ARRAY['extra.property', 'extra.property'], ARRAY['true', 'false']))", - "Invalid value for catalog 'hive' table property 'extra_properties': Cannot convert.*"); + "line 1:88: Invalid value for catalog 'hive' table property 'extra_properties': Cannot convert.*"); } @Test @@ -9130,7 +9130,7 @@ public void testDuplicateExtraPropertiesOnView() { assertQueryFails( "CREATE VIEW create_view_with_duplicate_extra_properties WITH (extra_properties = MAP(ARRAY['extra.property', 'extra.property'], ARRAY['true', 'false'])) AS SELECT 1 as colA", - "Invalid value for catalog 'hive' view property 'extra_properties': Cannot convert.*"); + "line 1:63: Invalid value for catalog 'hive' view property 'extra_properties': Cannot convert.*"); } @Test @@ -9159,7 +9159,7 @@ public void testCreateViewWithTableProperties() { assertQueryFails( "CREATE VIEW create_view_with_table_properties WITH (format = 'ORC', extra_properties = MAP(ARRAY['extra.property'], ARRAY['true'])) AS SELECT 1 as colA", - "Catalog 'hive' view property 'format' does not exist"); + "line 1:53: Catalog 'hive' view property 'format' does not exist"); } @Test diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index d7e809e591bf13..ecbc215415d79a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -5339,10 +5339,10 @@ public void testOptimizeParameterValidation() "\\Qline 1:7: Table 'iceberg.tpch.no_such_table_exists' does not exist"); assertQueryFails( "ALTER TABLE nation EXECUTE OPTIMIZE (file_size_threshold => '33')", - "\\QUnable to set catalog 'iceberg' 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 'iceberg' 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 'iceberg' table procedure 'OPTIMIZE' property 'file_size_threshold' to ['33s']: Unknown unit: s"); + "\\Qline 1:38: Unable to set catalog 'iceberg' table procedure 'OPTIMIZE' property 'file_size_threshold' to ['33s']: Unknown unit: s"); } @Test @@ -5940,10 +5940,10 @@ public void testExpireSnapshotsParameterValidation() "\\Qline 1:7: Table 'iceberg.tpch.no_such_table_exists' does not exist"); assertQueryFails( "ALTER TABLE nation EXECUTE EXPIRE_SNAPSHOTS (retention_threshold => '33')", - "\\QUnable to set catalog 'iceberg' table procedure 'EXPIRE_SNAPSHOTS' property 'retention_threshold' to ['33']: duration is not a valid data duration string: 33"); + "\\Qline 1:46: Unable to set catalog 'iceberg' table procedure 'EXPIRE_SNAPSHOTS' property 'retention_threshold' to ['33']: duration is not a valid data duration string: 33"); assertQueryFails( "ALTER TABLE nation EXECUTE EXPIRE_SNAPSHOTS (retention_threshold => '33mb')", - "\\QUnable to set catalog 'iceberg' table procedure 'EXPIRE_SNAPSHOTS' property 'retention_threshold' to ['33mb']: Unknown time unit: mb"); + "\\Qline 1:46: Unable to set catalog 'iceberg' table procedure 'EXPIRE_SNAPSHOTS' property 'retention_threshold' to ['33mb']: Unknown time unit: mb"); assertQueryFails( "ALTER TABLE nation EXECUTE EXPIRE_SNAPSHOTS (retention_threshold => '33s')", "\\QRetention specified (33.00s) is shorter than the minimum retention configured in the system (7.00d). Minimum retention can be changed with iceberg.expire_snapshots.min-retention configuration property or iceberg.expire_snapshots_min_retention session property"); @@ -6078,10 +6078,10 @@ public void testRemoveOrphanFilesParameterValidation() "\\Qline 1:7: Table 'iceberg.tpch.no_such_table_exists' does not exist"); assertQueryFails( "ALTER TABLE nation EXECUTE REMOVE_ORPHAN_FILES (retention_threshold => '33')", - "\\QUnable to set catalog 'iceberg' table procedure 'REMOVE_ORPHAN_FILES' property 'retention_threshold' to ['33']: duration is not a valid data duration string: 33"); + "\\Qline 1:49: Unable to set catalog 'iceberg' table procedure 'REMOVE_ORPHAN_FILES' property 'retention_threshold' to ['33']: duration is not a valid data duration string: 33"); assertQueryFails( "ALTER TABLE nation EXECUTE REMOVE_ORPHAN_FILES (retention_threshold => '33mb')", - "\\QUnable to set catalog 'iceberg' table procedure 'REMOVE_ORPHAN_FILES' property 'retention_threshold' to ['33mb']: Unknown time unit: mb"); + "\\Qline 1:49: Unable to set catalog 'iceberg' table procedure 'REMOVE_ORPHAN_FILES' property 'retention_threshold' to ['33mb']: Unknown time unit: mb"); assertQueryFails( "ALTER TABLE nation EXECUTE REMOVE_ORPHAN_FILES (retention_threshold => '33s')", "\\QRetention specified (33.00s) is shorter than the minimum retention configured in the system (7.00d). Minimum retention can be changed with iceberg.remove_orphan_files.min-retention configuration property or iceberg.remove_orphan_files_min_retention session property"); @@ -6153,7 +6153,7 @@ public void testUpdatingInvalidTableProperty() String tableName = "test_updating_invalid_table_property_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (a INT, b INT)"); assertThat(query("ALTER TABLE " + tableName + " SET PROPERTIES not_a_valid_table_property = 'a value'")) - .failure().hasMessage("Catalog 'iceberg' table property 'not_a_valid_table_property' does not exist"); + .failure().hasMessage("line 1:76: Catalog 'iceberg' table property 'not_a_valid_table_property' does not exist"); assertUpdate("DROP TABLE " + tableName); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java index 31137cd8f45473..f0f066ce3ccd23 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java @@ -177,7 +177,7 @@ public void testCreateWithInvalidPropertyFails() assertThatThrownBy(() -> computeActual("CREATE MATERIALIZED VIEW materialized_view_with_property " + "WITH (invalid_property = ARRAY['_date']) AS " + "SELECT _bigint, _date FROM base_table1")) - .hasMessage("Catalog 'iceberg' materialized view property 'invalid_property' does not exist"); + .hasMessage("line 1:64: Catalog 'iceberg' materialized view property 'invalid_property' does not exist"); } @Test diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java index 17838794dab767..6b87934499f48d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java @@ -497,7 +497,7 @@ public void testAnalyzeSomeColumns() assertUpdate(noStatsOnWrite, "CREATE TABLE " + tableName + " AS SELECT * FROM tpch.sf1.nation", 25); // analyze NULL list of columns - assertQueryFails("ANALYZE " + tableName + " WITH (columns = NULL)", "\\QInvalid null value for catalog 'iceberg' analyze property 'columns' from [null]"); + assertQueryFails("ANALYZE " + tableName + " WITH (columns = NULL)", "\\Qline 1:41: Invalid null value for catalog 'iceberg' analyze property 'columns' from [null]"); // analyze empty list of columns assertQueryFails("ANALYZE " + tableName + " WITH (columns = ARRAY[])", "\\QCannot specify empty list of columns for analysis"); @@ -511,7 +511,7 @@ public void testAnalyzeSomeColumns() // specify NULL column assertQueryFails( "ANALYZE " + tableName + " WITH (columns = ARRAY['nationkey', NULL])", - "\\QUnable to set catalog 'iceberg' analyze property 'columns' to [ARRAY['nationkey',null]]: Invalid null value in analyze columns property"); + "\\Qline 1:41: Unable to set catalog 'iceberg' analyze property 'columns' to [ARRAY['nationkey',null]]: Invalid null value in analyze columns property"); // analyze nationkey and regionkey assertUpdate("ANALYZE " + tableName + " WITH (columns = ARRAY['nationkey', 'regionkey'])"); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java index b77be164b9393a..9327478a493b9b 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java @@ -589,7 +589,7 @@ public void testUpgradingToInvalidVersionFails() assertUpdate("CREATE TABLE " + tableName + " WITH (format_version = 2) AS SELECT * FROM tpch.tiny.nation", 25); assertThat(loadTable(tableName).operations().current().formatVersion()).isEqualTo(2); assertThat(query("ALTER TABLE " + tableName + " SET PROPERTIES format_version = 42")) - .failure().hasMessage("Unable to set catalog 'iceberg' table property 'format_version' to [42]: format_version must be between 1 and 2"); + .failure().hasMessage("line 1:79: Unable to set catalog 'iceberg' table property 'format_version' to [42]: format_version must be between 1 and 2"); } @Test diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java b/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java index b97cb96d3908e1..7dc57c38fa77cf 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java @@ -248,7 +248,7 @@ public void testSchemaProperties() { assertUpdate("CREATE SCHEMA mock.test_schema WITH (boolean_schema_property = true)"); assertThatThrownBy(() -> assertUpdate("CREATE SCHEMA mock.test_schema WITH (unknown_property = true)")) - .hasMessage("Catalog 'mock' schema property 'unknown_property' does not exist"); + .hasMessage("line 1:38: Catalog 'mock' schema property 'unknown_property' does not exist"); } @Test @@ -266,6 +266,6 @@ public void testTableProperties() { assertUpdate("CREATE TABLE mock.default.new_table (c int) WITH (integer_table_property = 1)"); assertThatThrownBy(() -> assertUpdate("CREATE TABLE mock.default.new_table (c int) WITH (unknown_property = 1)")) - .hasMessage("Catalog 'mock' table property 'unknown_property' does not exist"); + .hasMessage("line 1:51: Catalog 'mock' table property 'unknown_property' does not exist"); } } diff --git a/testing/trino-tests/src/test/java/io/trino/tests/tpch/TestTpchConnectorTest.java b/testing/trino-tests/src/test/java/io/trino/tests/tpch/TestTpchConnectorTest.java index ffd4b7ff061418..819d4fb65d668a 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/tpch/TestTpchConnectorTest.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/tpch/TestTpchConnectorTest.java @@ -120,7 +120,7 @@ public void testAnalyzePropertiesSystemTable() public void testAnalyze() { assertUpdate("ANALYZE orders", 15000); - assertQueryFails("ANALYZE orders WITH (foo = 'bar')", ".* analyze property 'foo' does not exist"); + assertQueryFails("ANALYZE orders WITH (foo = 'bar')", "line 1:22: Catalog 'tpch' analyze property 'foo' does not exist"); } @Test