Skip to content

Commit 1ae19f5

Browse files
committed
Add support for truncate statement in Iceberg
1 parent 1e2d683 commit 1ae19f5

File tree

3 files changed

+23
-5
lines changed

3 files changed

+23
-5
lines changed

plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -306,6 +306,7 @@
306306
import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL_DEFAULT;
307307
import static org.apache.iceberg.TableProperties.FORMAT_VERSION;
308308
import static org.apache.iceberg.TableProperties.WRITE_LOCATION_PROVIDER_IMPL;
309+
import static org.apache.iceberg.expressions.Expressions.alwaysTrue;
309310
import static org.apache.iceberg.types.TypeUtil.indexParents;
310311
import static org.apache.iceberg.util.LocationUtil.stripTrailingSlash;
311312
import static org.apache.iceberg.util.SnapshotUtil.schemaFor;
@@ -2551,6 +2552,16 @@ public OptionalLong executeDelete(ConnectorSession session, ConnectorTableHandle
25512552
return OptionalLong.of(deletedRecords - removedPositionDeletes - removedEqualityDeletes);
25522553
}
25532554

2555+
@Override
2556+
public void truncateTable(ConnectorSession session, ConnectorTableHandle tableHandle)
2557+
{
2558+
IcebergTableHandle table = checkValidTableHandle(tableHandle);
2559+
Table icebergTable = catalog.loadTable(session, table.getSchemaTableName());
2560+
DeleteFiles deleteFiles = icebergTable.newDelete()
2561+
.deleteFromRowFilter(alwaysTrue());
2562+
commit(deleteFiles, session);
2563+
}
2564+
25542565
public void rollback()
25552566
{
25562567
// TODO: cleanup open transaction

plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -84,8 +84,7 @@ public void initFileSystem()
8484
protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
8585
{
8686
return switch (connectorBehavior) {
87-
case SUPPORTS_TOPN_PUSHDOWN,
88-
SUPPORTS_TRUNCATE -> false;
87+
case SUPPORTS_TOPN_PUSHDOWN -> false;
8988
default -> super.hasBehavior(connectorBehavior);
9089
};
9190
}

plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -225,8 +225,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
225225
SUPPORTS_REPORTING_WRITTEN_BYTES -> true;
226226
case SUPPORTS_ADD_COLUMN_NOT_NULL_CONSTRAINT,
227227
SUPPORTS_RENAME_MATERIALIZED_VIEW_ACROSS_SCHEMAS,
228-
SUPPORTS_TOPN_PUSHDOWN,
229-
SUPPORTS_TRUNCATE -> false;
228+
SUPPORTS_TOPN_PUSHDOWN -> false;
230229
default -> super.hasBehavior(connectorBehavior);
231230
};
232231
}
@@ -6227,6 +6226,15 @@ public void testEmptyDelete()
62276226
assertUpdate("DROP TABLE " + tableName);
62286227
}
62296228

6229+
@Test
6230+
public void testEmptyFilesTruncate()
6231+
{
6232+
try (TestTable table = new TestTable(getQueryRunner()::execute, "test_empty_files_truncate_", "AS SELECT 1 AS id")) {
6233+
assertUpdate("TRUNCATE TABLE " + table.getName());
6234+
assertQueryReturnsEmptyResult("SELECT * FROM \"" + table.getName() + "$files\"");
6235+
}
6236+
}
6237+
62306238
@Test
62316239
public void testModifyingOldSnapshotIsNotPossible()
62326240
{
@@ -7298,7 +7306,7 @@ public void testCorruptedTableLocation()
72987306
assertQueryFails("UPDATE " + tableName + " SET country = 'AUSTRIA'", "Metadata not found in metadata location for table " + schemaTableName);
72997307
assertQueryFails("DELETE FROM " + tableName, "Metadata not found in metadata location for table " + schemaTableName);
73007308
assertQueryFails("MERGE INTO " + tableName + " USING (SELECT 1 a) input ON true WHEN MATCHED THEN DELETE", "Metadata not found in metadata location for table " + schemaTableName);
7301-
assertQueryFails("TRUNCATE TABLE " + tableName, "This connector does not support truncating tables");
7309+
assertQueryFails("TRUNCATE TABLE " + tableName, "Metadata not found in metadata location for table " + schemaTableName);
73027310
assertQueryFails("COMMENT ON TABLE " + tableName + " IS NULL", "Metadata not found in metadata location for table " + schemaTableName);
73037311
assertQueryFails("COMMENT ON COLUMN " + tableName + ".foo IS NULL", "Metadata not found in metadata location for table " + schemaTableName);
73047312
assertQueryFails("CALL iceberg.system.rollback_to_snapshot(CURRENT_SCHEMA, '" + tableName + "', 8954597067493422955)", "Metadata not found in metadata location for table " + schemaTableName);

0 commit comments

Comments
 (0)