From 76fb5ebe420df68b0e0832e1ce2ec5db52aa81a6 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 13 Apr 2023 13:26:01 +0900 Subject: [PATCH] fixup! Support DML operations on Delta tables with `name` column mapping --- .../plugin/deltalake/DeltaLakeMetadata.java | 72 +++++++----- .../TestDeltaLakeAlterTableCompatibility.java | 39 +++++-- .../TestDeltaLakeColumnMappingMode.java | 103 +++++++++++++++++- 3 files changed, 178 insertions(+), 36 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java index 438919f05ae5..7437bb8b755c 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java @@ -194,6 +194,7 @@ import static io.trino.plugin.deltalake.metastore.HiveMetastoreBackedDeltaLakeMetastore.TABLE_PROVIDER_VALUE; import static io.trino.plugin.deltalake.procedure.DeltaLakeTableProcedureId.OPTIMIZE; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.APPEND_ONLY_CONFIGURATION_KEY; +import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.ColumnMappingMode.NONE; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.changeDataFeedEnabled; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.extractPartitionColumns; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.extractSchema; @@ -287,9 +288,8 @@ public class DeltaLakeMetadata public static final int DEFAULT_WRITER_VERSION = 2; // The highest reader and writer versions Trino supports private static final int MAX_READER_VERSION = 3; - private static final int MAX_WRITER_VERSION = 4; + private static final int MAX_WRITER_VERSION = 5; private static final int CDF_SUPPORTED_WRITER_VERSION = 4; - private static final int MAX_DML_WRITER_VERSION = 5; // Matches the dummy column Databricks stores in the metastore private static final List DUMMY_DATA_COLUMNS = ImmutableList.of( @@ -1064,6 +1064,11 @@ public void setTableComment(ConnectorSession session, ConnectorTableHandle table { DeltaLakeTableHandle handle = checkValidTableHandle(tableHandle); checkSupportedWriterVersion(session, handle.getSchemaTableName()); + ColumnMappingMode columnMappingMode = getColumnMappingMode(handle.getMetadataEntry()); + if (columnMappingMode != NONE) { + // TODO https://github.com/trinodb/trino/issues/12638 Support setting a table comment for id and name column mapping mode + throw new TrinoException(NOT_SUPPORTED, "Setting a table comment with column mapping %s is not supported".formatted(columnMappingMode.name().toLowerCase(ENGLISH))); + } ConnectorTableMetadata tableMetadata = getTableMetadata(session, handle); @@ -1104,6 +1109,11 @@ public void setColumnComment(ConnectorSession session, ConnectorTableHandle tabl DeltaLakeTableHandle deltaLakeTableHandle = (DeltaLakeTableHandle) tableHandle; DeltaLakeColumnHandle deltaLakeColumnHandle = (DeltaLakeColumnHandle) column; checkSupportedWriterVersion(session, deltaLakeTableHandle.getSchemaTableName()); + ColumnMappingMode columnMappingMode = getColumnMappingMode(deltaLakeTableHandle.getMetadataEntry()); + if (columnMappingMode != NONE) { + // TODO https://github.com/trinodb/trino/issues/12638 Support setting a column comment for id and name column mapping mode + throw new TrinoException(NOT_SUPPORTED, "Setting a column comment with column mapping %s is not supported".formatted(columnMappingMode.name().toLowerCase(ENGLISH))); + } ConnectorTableMetadata tableMetadata = getTableMetadata(session, deltaLakeTableHandle); @@ -1149,6 +1159,11 @@ public void addColumn(ConnectorSession session, ConnectorTableHandle tableHandle { DeltaLakeTableHandle handle = checkValidTableHandle(tableHandle); checkSupportedWriterVersion(session, handle.getSchemaTableName()); + ColumnMappingMode columnMappingMode = getColumnMappingMode(handle.getMetadataEntry()); + if (columnMappingMode != NONE) { + // TODO https://github.com/trinodb/trino/issues/12638 Support adding a column for id and name column mapping mode + throw new TrinoException(NOT_SUPPORTED, "Adding a column with column mapping %s is not supported".formatted(columnMappingMode.name().toLowerCase(ENGLISH))); + } if (changeDataFeedEnabled(handle.getMetadataEntry()) && CHANGE_DATA_FEED_COLUMN_NAMES.contains(newColumnMetadata.getName())) { throw new TrinoException(NOT_SUPPORTED, "Column name %s is forbidden when change data feed is enabled".formatted(newColumnMetadata.getName())); } @@ -1271,9 +1286,7 @@ public ConnectorInsertTableHandle beginInsert(ConnectorSession session, Connecto format("Inserts are not enabled on the %1$s filesystem in order to avoid eventual data corruption which may be caused by concurrent data modifications on the table. " + "Writes to the %1$s filesystem can be however enabled with the '%2$s' configuration property.", fileSystem, ENABLE_NON_CONCURRENT_WRITES_CONFIGURATION_KEY)); } - checkUnsupportedGeneratedColumns(table.getMetadataEntry()); - checkUnsupportedDmlColumnMapping(table.getMetadataEntry()); - checkSupportedDmlWriterVersion(session, table.getSchemaTableName()); + checkUnsupportedWriter(session, table.getSchemaTableName(), table.getMetadataEntry()); List inputColumns = columns.stream() .map(handle -> (DeltaLakeColumnHandle) handle) @@ -1445,9 +1458,7 @@ public ConnectorMergeTableHandle beginMerge(ConnectorSession session, ConnectorT if (!getCheckConstraints(handle.getMetadataEntry()).isEmpty()) { throw new TrinoException(NOT_SUPPORTED, "Writing to tables with CHECK constraints is not supported"); } - checkUnsupportedGeneratedColumns(handle.getMetadataEntry()); - checkUnsupportedDmlColumnMapping(handle.getMetadataEntry()); - checkSupportedDmlWriterVersion(session, handle.getSchemaTableName()); + checkUnsupportedWriter(session, handle.getSchemaTableName(), handle.getMetadataEntry()); ConnectorTableMetadata tableMetadata = getTableMetadata(session, handle); @@ -1664,6 +1675,11 @@ private BeginTableExecuteResult( executeHandle.withProcedureHandle(optimizeHandle.withCurrentVersion(table.getReadVersion())), @@ -1754,29 +1770,22 @@ private boolean allowWrite(ConnectorSession session, DeltaLakeTableHandle tableH } } - private void checkUnsupportedGeneratedColumns(MetadataEntry metadataEntry) - { - Map columnGeneratedExpressions = getGeneratedColumnExpressions(metadataEntry); - if (!columnGeneratedExpressions.isEmpty()) { - throw new TrinoException(NOT_SUPPORTED, "Writing to tables with generated columns is not supported"); - } - } - - private void checkUnsupportedDmlColumnMapping(MetadataEntry metadataEntry) + private void checkUnsupportedWriter(ConnectorSession session, SchemaTableName schemaTableName, MetadataEntry metadataEntry) { + checkSupportedWriterVersion(session, schemaTableName); + checkUnsupportedGeneratedColumns(metadataEntry); ColumnMappingMode columnMappingMode = getColumnMappingMode(metadataEntry); - if (!(columnMappingMode == ColumnMappingMode.NONE || columnMappingMode == ColumnMappingMode.NAME)) { + if (!(columnMappingMode == NONE || columnMappingMode == ColumnMappingMode.NAME)) { throw new TrinoException(NOT_SUPPORTED, "Writing with column mapping %s is not supported".formatted(columnMappingMode.name().toLowerCase(ENGLISH))); } + // TODO: Check writer-features } - private void checkSupportedDmlWriterVersion(ConnectorSession session, SchemaTableName schemaTableName) + private void checkUnsupportedGeneratedColumns(MetadataEntry metadataEntry) { - int requiredWriterVersion = getProtocolEntry(session, schemaTableName).getMinWriterVersion(); - if (requiredWriterVersion > MAX_DML_WRITER_VERSION) { - throw new TrinoException( - NOT_SUPPORTED, - format("Table %s requires Delta Lake writer version %d which is not supported", schemaTableName, requiredWriterVersion)); + Map columnGeneratedExpressions = getGeneratedColumnExpressions(metadataEntry); + if (!columnGeneratedExpressions.isEmpty()) { + throw new TrinoException(NOT_SUPPORTED, "Writing to tables with generated columns is not supported"); } } @@ -2399,7 +2408,12 @@ private void updateTableStatistics( Map mergedColumnStatistics = newColumnStatistics.entrySet().stream() .map(entry -> { String columnName = entry.getKey(); - return Map.entry(physicalColumnNameMapping.map(mapping -> mapping.get(columnName)).orElse(columnName), entry.getValue()); + String physicalColumnName = columnName; + if (physicalColumnNameMapping.isPresent()) { + physicalColumnName = physicalColumnNameMapping.get().get(columnName); + requireNonNull(physicalColumnName, () -> "%s doesn't exist in %s".formatted(columnName, physicalColumnNameMapping)); + } + return Map.entry(physicalColumnName, entry.getValue()); }) .collect(toImmutableMap( Entry::getKey, @@ -2431,7 +2445,13 @@ private void updateTableStatistics( analyzedColumns.ifPresent(analyzeColumns -> { Set analyzePhysicalColumns = analyzeColumns.stream() - .map(columnName -> physicalColumnNameMapping.map(mapping -> mapping.get(columnName)).orElse(columnName)) + .map(columnName -> { + if (physicalColumnNameMapping.isPresent()) { + String physicalColumnName = physicalColumnNameMapping.get().get(columnName); + return requireNonNull(physicalColumnName, () -> "%s doesn't exist in %s".formatted(columnName, physicalColumnNameMapping)); + } + return columnName; + }) .collect(toImmutableSet()); if (!mergedColumnStatistics.keySet().equals(analyzePhysicalColumns)) { // sanity validation diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeAlterTableCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeAlterTableCompatibility.java index a39b6b77e273..3ea113af405b 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeAlterTableCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeAlterTableCompatibility.java @@ -27,6 +27,7 @@ import static io.trino.tempto.assertions.QueryAssert.assertThat; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; +import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_104; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_73; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_91; import static io.trino.tests.product.TestGroups.DELTA_LAKE_OSS; @@ -71,7 +72,7 @@ public void testAddColumnWithCommentOnTrino() } } - @Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, DELTA_LAKE_EXCLUDE_73, PROFILE_SPECIFIC_TESTS}) + @Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, DELTA_LAKE_EXCLUDE_73, DELTA_LAKE_EXCLUDE_91, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testAddColumnUnsupportedWriterVersion() { @@ -81,14 +82,14 @@ public void testAddColumnUnsupportedWriterVersion() onDelta().executeQuery(format("" + "CREATE TABLE default.%s (col int) " + "USING DELTA LOCATION 's3://%s/%s'" + - "TBLPROPERTIES ('delta.minWriterVersion'='5')", + "TBLPROPERTIES ('delta.minWriterVersion'='6')", tableName, bucketName, tableDirectory)); try { assertQueryFailure(() -> onTrino().executeQuery("ALTER TABLE delta.default." + tableName + " ADD COLUMN new_col int")) - .hasMessageMatching(".* Table .* requires Delta Lake writer version 5 which is not supported"); + .hasMessageMatching(".* Table .* requires Delta Lake writer version 6 which is not supported"); } finally { dropDeltaTableWithRetry("default." + tableName); @@ -196,14 +197,14 @@ public void testCommentOnTableUnsupportedWriterVersion() onDelta().executeQuery(format("" + "CREATE TABLE default.%s (col int) " + "USING DELTA LOCATION 's3://%s/%s'" + - "TBLPROPERTIES ('delta.minWriterVersion'='5')", + "TBLPROPERTIES ('delta.minWriterVersion'='6')", tableName, bucketName, tableDirectory)); try { assertQueryFailure(() -> onTrino().executeQuery("COMMENT ON TABLE delta.default." + tableName + " IS 'test comment'")) - .hasMessageMatching(".* Table .* requires Delta Lake writer version 5 which is not supported"); + .hasMessageMatching(".* Table .* requires Delta Lake writer version 6 which is not supported"); } finally { onTrino().executeQuery("DROP TABLE delta.default." + tableName); @@ -242,20 +243,44 @@ public void testCommentOnColumnUnsupportedWriterVersion() onDelta().executeQuery(format("" + "CREATE TABLE default.%s (col int) " + "USING DELTA LOCATION 's3://%s/%s'" + - "TBLPROPERTIES ('delta.minWriterVersion'='5')", + "TBLPROPERTIES ('delta.minWriterVersion'='6')", tableName, bucketName, tableDirectory)); try { assertQueryFailure(() -> onTrino().executeQuery("COMMENT ON COLUMN delta.default." + tableName + ".col IS 'test column comment'")) - .hasMessageMatching(".* Table .* requires Delta Lake writer version 5 which is not supported"); + .hasMessageMatching(".* Table .* requires Delta Lake writer version 6 which is not supported"); } finally { onTrino().executeQuery("DROP TABLE delta.default." + tableName); } } + @Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, DELTA_LAKE_EXCLUDE_73, DELTA_LAKE_EXCLUDE_91, DELTA_LAKE_EXCLUDE_104, PROFILE_SPECIFIC_TESTS}) + @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) + public void testOptimizeUnsupportedWriterVersion() + { + String tableName = "test_dl_optimize_unsupported_writer_" + randomNameSuffix(); + String tableDirectory = "databricks-compatibility-test-" + tableName; + + onDelta().executeQuery(format("" + + "CREATE TABLE default.%s (col int) " + + "USING DELTA LOCATION 's3://%s/%s'" + + "TBLPROPERTIES ('delta.minWriterVersion'='6')", + tableName, + bucketName, + tableDirectory)); + + try { + assertQueryFailure(() -> onTrino().executeQuery("ALTER TABLE delta.default." + tableName + " EXECUTE OPTIMIZE")) + .hasMessageMatching(".* Table .* requires Delta Lake writer version 6 which is not supported"); + } + finally { + dropDeltaTableWithRetry(tableName); + } + } + @Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoAlterTablePreservesTableMetadata() diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeColumnMappingMode.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeColumnMappingMode.java index 2ab643f86445..d02fa1624995 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeColumnMappingMode.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeColumnMappingMode.java @@ -28,6 +28,7 @@ import static io.trino.testing.DataProviders.trueFalse; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; +import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_104; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_73; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_91; import static io.trino.tests.product.TestGroups.DELTA_LAKE_OSS; @@ -330,7 +331,7 @@ public void testShowStatsOnPartitionedForColumnMappingMode(String mode) } } - @Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, DELTA_LAKE_EXCLUDE_73, DELTA_LAKE_EXCLUDE_91, PROFILE_SPECIFIC_TESTS}, dataProvider = "columnMappingDataProvider") + @Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, DELTA_LAKE_EXCLUDE_73, DELTA_LAKE_EXCLUDE_91, DELTA_LAKE_EXCLUDE_104, PROFILE_SPECIFIC_TESTS}, dataProvider = "columnMappingDataProvider") @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDropAndAddColumnShowStatsForColumnMappingMode(String mode) { @@ -375,6 +376,98 @@ public void testDropAndAddColumnShowStatsForColumnMappingMode(String mode) } } + @Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, DELTA_LAKE_EXCLUDE_73, DELTA_LAKE_EXCLUDE_91, DELTA_LAKE_EXCLUDE_104, PROFILE_SPECIFIC_TESTS}) + @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) + public void testChangeColumnMappingAndShowStatsForColumnMappingMode() + { + String tableName = "test_dl_change_column_mapping_and_show_stats_for_column_mapping_mode_" + randomNameSuffix(); + + onDelta().executeQuery("" + + "CREATE TABLE default." + tableName + + " (a_number INT, b_number INT)" + + " USING delta " + + " LOCATION 's3://" + bucketName + "/databricks-compatibility-test-" + tableName + "'" + + " TBLPROPERTIES (" + + " 'delta.columnMapping.mode'='none'," + + " 'delta.minReaderVersion'='2'," + + " 'delta.minWriterVersion'='5')"); + try { + onDelta().executeQuery("INSERT INTO default." + tableName + " VALUES (1, 10), (2, 20), (null, null)"); + onTrino().executeQuery("ANALYZE delta.default." + tableName); + assertThat(onTrino().executeQuery("SHOW STATS FOR delta.default." + tableName)) + .containsOnly( + row("a_number", null, 2.0, 0.33333333333, null, "1", "2"), + row("b_number", null, 2.0, 0.33333333333, null, "10", "20"), + row(null, null, null, null, 3.0, null, null)); + + onDelta().executeQuery("ALTER TABLE default." + tableName + " SET TBLPROPERTIES('delta.columnMapping.mode'='name')"); + onDelta().executeQuery("ALTER TABLE default." + tableName + " DROP COLUMN b_number"); + onDelta().executeQuery("ALTER TABLE default." + tableName + " ADD COLUMN b_number INT"); + + // Ensure SHOW STATS doesn't return stats for the restored column + assertThat(onTrino().executeQuery("SHOW STATS FOR delta.default." + tableName)) + .containsOnly( + row("a_number", null, 2.0, 0.33333333333, null, "1", "2"), + row("b_number", null, null, null, null, null, null), + row(null, null, null, null, 3.0, null, null)); + + // SHOW STATS returns the expected stats after executing ANALYZE + onTrino().executeQuery("ANALYZE delta.default." + tableName); + assertThat(onTrino().executeQuery("SHOW STATS FOR delta.default." + tableName)) + .containsOnly( + row("a_number", null, 2.0, 0.33333333333, null, "1", "2"), + row("b_number", 0.0, 0.0, 1.0, null, null, null), + row(null, null, null, null, 3.0, null, null)); + } + finally { + dropDeltaTableWithRetry("default." + tableName); + } + } + + @Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, DELTA_LAKE_EXCLUDE_73, DELTA_LAKE_EXCLUDE_91, PROFILE_SPECIFIC_TESTS}, dataProvider = "changeColumnMappingDataProvider") + @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) + public void testChangeColumnMappingMode(String sourceMappingMode, String targetMappingMode, boolean supported) + { + String tableName = "test_dl_change_column_mapping_mode_" + randomNameSuffix(); + + onDelta().executeQuery("" + + "CREATE TABLE default." + tableName + + " (a_number INT)" + + " USING delta " + + " LOCATION 's3://" + bucketName + "/databricks-compatibility-test-" + tableName + "'" + + " TBLPROPERTIES (" + + " 'delta.columnMapping.mode'='" + sourceMappingMode + "'," + + " 'delta.minReaderVersion'='2'," + + " 'delta.minWriterVersion'='5')"); + try { + if (supported) { + onDelta().executeQuery("ALTER TABLE default." + tableName + " SET TBLPROPERTIES('delta.columnMapping.mode'='" + targetMappingMode + "')"); + } + else { + assertQueryFailure(() -> onDelta().executeQuery("ALTER TABLE default." + tableName + " SET TBLPROPERTIES('delta.columnMapping.mode'='" + targetMappingMode + "')")) + .hasMessageContaining("Changing column mapping mode from '%s' to '%s' is not supported".formatted(sourceMappingMode, targetMappingMode)); + } + } + finally { + dropDeltaTableWithRetry("default." + tableName); + } + } + + @DataProvider + public Object[][] changeColumnMappingDataProvider() + { + // Update testChangeColumnMappingAndShowStatsForColumnMappingMode if Delta Lake changes their behavior + return new Object[][] { + // sourceMappingMode targetMappingMode supported + {"none", "id", false}, + {"none", "name", true}, + {"id", "none", false}, + {"id", "name", false}, + {"name", "none", false}, + {"name", "id", false}, + }; + } + @Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, DELTA_LAKE_EXCLUDE_73, DELTA_LAKE_EXCLUDE_91, PROFILE_SPECIFIC_TESTS}, dataProvider = "columnMappingDataProvider") @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testUnsupportedOperationsColumnMappingMode(String mode) @@ -408,10 +501,14 @@ public void testUnsupportedOperationsColumnMappingMode(String mode) assertQueryFailure(() -> onTrino().executeQuery("UPDATE delta.default." + tableName + " SET a_string = 'test'")) .hasMessageContaining("Writing with column mapping id is not supported"); } + assertQueryFailure(() -> onTrino().executeQuery("COMMENT ON TABLE delta.default." + tableName + " IS 'test comment'")) + .hasMessageContaining("Setting a table comment with column mapping %s is not supported".formatted(mode)); + assertQueryFailure(() -> onTrino().executeQuery("COMMENT ON COLUMN delta.default." + tableName + ".a_number IS 'test comment'")) + .hasMessageContaining("Setting a column comment with column mapping %s is not supported".formatted(mode)); assertQueryFailure(() -> onTrino().executeQuery("ALTER TABLE delta.default." + tableName + " EXECUTE OPTIMIZE")) - .hasMessageContaining("Delta Lake writer version 5 which is not supported"); + .hasMessageContaining("Executing 'optimize' procedure with column mapping %s is not supported".formatted(mode)); assertQueryFailure(() -> onTrino().executeQuery("ALTER TABLE delta.default." + tableName + " ADD COLUMN new_col varchar")) - .hasMessageContaining("Delta Lake writer version 5 which is not supported"); + .hasMessageContaining("Adding a column with column mapping %s is not supported".formatted(mode)); assertQueryFailure(() -> onTrino().executeQuery("ALTER TABLE delta.default." + tableName + " RENAME COLUMN a_number TO renamed_column")) .hasMessageContaining("This connector does not support renaming columns"); assertQueryFailure(() -> onTrino().executeQuery("ALTER TABLE delta.default." + tableName + " DROP COLUMN a_number"))