Skip to content

Conversation

@ebyhr
Copy link
Member

@ebyhr ebyhr commented Nov 19, 2025

Description

The getTableHandle method returns null if the table is dropped concurrently during the loop.

Release notes

## Iceberg
* Fix potential failure when dropping a schema with cascade. ({issue}`27361`)

@cla-bot cla-bot bot added the cla-signed label Nov 19, 2025
@github-actions github-actions bot added the iceberg Iceberg connector label Nov 19, 2025
for (SchemaTableName tableName : listTables(session, Optional.of(schemaName))) {
dropTable(session, getTableHandle(session, tableName, Optional.empty(), Optional.empty()));
ConnectorTableHandle tableHandle = getTableHandle(session, tableName, Optional.empty(), Optional.empty());
if (tableHandle != null) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Worth code comment

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why view not have such issue?

Copy link
Member

@Math-ias Math-ias Nov 20, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reposting to main PR thread. I think views don't have that issue because it's a quick update to remove them from the metastore. Files connected to an iceberg table will take longer to delete and that increases risk of the table disappearing underneath.

for (SchemaTableName tableName : listTables(session, Optional.of(schemaName))) {
dropTable(session, getTableHandle(session, tableName, Optional.empty(), Optional.empty()));
ConnectorTableHandle tableHandle = getTableHandle(session, tableName, Optional.empty(), Optional.empty());
if (tableHandle != null) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

dropTable(session, getTableHandle(session, tableName, Optional.empty(), Optional.empty()));
ConnectorTableHandle tableHandle = getTableHandle(session, tableName, Optional.empty(), Optional.empty());
if (tableHandle != null) {
dropTable(session, tableHandle);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So to understand the motivation of the change correctly ... Looks like dropTable is potentially blocking (for a while) the thread?

following method calls in github

Therefore tables we've listed from metadata can get dropped from beneath this thread as a DROP SCHEMA schema_name CASCADE is completing ...

I'm guessing the risk of dropping a table taking so long is mostly when there's externally managed files. The delta lake connector should be comparable. In your implementation of CASCADE there in the delta lake connector you use an additional try on dropTable even after we've checked for tableHandle existence. Is that worth bringing back to this PR as well for extra safety?

Copy link
Member Author

@ebyhr ebyhr Nov 20, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(I didn't read this message carefully before merging the PR). Each Iceberg catalog maintains its own table metadata cache, so the first TrinoCatalog.loadTable call from IcebergMetadata.getTableHandle populates the cache, and the second TrinoCatalog.loadTable in TrinoCatalog.dropTable will retrieve the object from the cache. We can still add the catch block, though.

@ebyhr ebyhr force-pushed the ebi/iceberg-drop-schema-cascade branch from ca9055e to bfdf083 Compare November 20, 2025 03:56
@ebyhr ebyhr merged commit fb81bf5 into trinodb:master Nov 20, 2025
3 of 13 checks passed
@ebyhr ebyhr deleted the ebi/iceberg-drop-schema-cascade branch November 20, 2025 03:57
@github-actions github-actions bot added this to the 479 milestone Nov 20, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

cla-signed iceberg Iceberg connector

Development

Successfully merging this pull request may close these issues.

5 participants