Hater's guide to Iceberg - Part 1 - Deleting a table
In most database and data-lake systems, deleting a table is easy: DROP TABLE tbl_name and it's done… but under Iceberg it's more complicated, since it might leave behind stray files and directories.
Basic SQL approach
Every sample of code has been tested with JDK 17, Ammonite 3.0 and Scala 2.13. The libraries used are Spark 4.0.2 and Iceberg 1.10.1. The launch command is below.
JAVA_OPTS='--add-exports java.base/sun.nio.ch=ALL-UNNAMED' amm example.sc
First, the DROP TABLE command doesn't delete the data, it just deletes the
table's name from the table catalog, leaving the data-files untouched – this behavior
is similar to Spark's external table use-case. It's been
documented, and mostly true (more on that later).
import $ivy.`org.apache.spark::spark-core:4.0.2`
import $ivy.`org.apache.spark::spark-sql:4.0.2`
import $ivy.`org.apache.iceberg::iceberg-spark-runtime-4.0:1.10.1`
import $ivy.`org.xerial:sqlite-jdbc:3.51.1.0`
// Cleanup command:
// rm -rf database.db spark-warehouse/ warehouse/
import org.apache.spark.sql.SparkSession
@main
def main() : Unit = {
val spark = SparkSession
.builder()
.appName("Iceberg table deletion")
.master("local[*]")
.config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
.config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog")
.config("spark.sql.catalog.spark_catalog.type", "jdbc")
.config("spark.sql.catalog.spark_catalog.uri", "jdbc:sqlite:database.db")
.config("spark.sql.catalog.spark_catalog.warehouse", "warehouse")
.getOrCreate()
spark.sql("""
CREATE TABLE db.table
USING iceberg
PARTITIONED BY (bucket(16, id))
AS (
VALUES (22, 'aa'), (33, 'bb'), (44, 'cc')
AS sample_val (id, data)
)
""")
spark.sql("DROP TABLE db.table")
}
Result:
.
|-- database.db
|-- spark-warehouse
|-- test.sc
`-- warehouse
`-- db
`-- table
|-- data
| |-- id_bucket=1
| | `-- 00000-3-e97ebdb5-f2cb-4873-9641-5a0073ca6bcf-0-00002.parquet
| `-- id_bucket=13
| `-- 00000-3-e97ebdb5-f2cb-4873-9641-5a0073ca6bcf-0-00001.parquet
`-- metadata
|-- 00000-565b7429-b403-4f8b-b630-24492efaa7e2.metadata.json
|-- da88a3e1-04d4-4aee-844e-dc6887bde34c-m0.avro
`-- snap-1690246185083637803-1-da88a3e1-04d4-4aee-844e-dc6887bde34c.avro
Better SQL approach
In order to delete the tracked files belonging to a table, we have to run the
DROP TABLE tbl_name PURGE. This solves most of the problems…
import $ivy.`org.apache.spark::spark-core:4.0.2`
import $ivy.`org.apache.spark::spark-sql:4.0.2`
import $ivy.`org.apache.iceberg::iceberg-spark-runtime-4.0:1.10.1`
import $ivy.`org.xerial:sqlite-jdbc:3.51.1.0`
// Cleanup command:
// rm -rf database.db spark-warehouse/ warehouse/
import org.apache.spark.sql.SparkSession
@main
def main() : Unit = {
val spark = SparkSession
.builder()
.appName("Iceberg table deletion")
.master("local[*]")
.config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
.config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog")
.config("spark.sql.catalog.spark_catalog.type", "jdbc")
.config("spark.sql.catalog.spark_catalog.uri", "jdbc:sqlite:database.db")
.config("spark.sql.catalog.spark_catalog.warehouse", "warehouse")
.getOrCreate()
spark.sql("""
CREATE TABLE db.table
USING iceberg
PARTITIONED BY (bucket(16, id))
AS (
VALUES (22, 'aa'), (33, 'bb'), (44, 'cc')
AS sample_val (id, data)
)
""")
spark.sql("DROP TABLE db.table PURGE")
}
Result:
.
|-- database.db
|-- spark-warehouse
|-- test.sc
`-- warehouse
`-- db
`-- table
|-- data
| |-- id_bucket=1
| `-- id_bucket=13
`-- metadata
We see that, yes, the files have been deleted… but the directories remain. This is a leaky abstraction where the complexity of the storage layer is not completely handled. If the storage layer for the data-files uses object-storage semantics ("objects are binary blobs in a key-value storage, directories aren't real") it works fine, but if the storage layer is a filesystem (and empty directories can exist, such as in local filesystems or HDFS), we're left with a lot of empty directories. This is a known & ignored issue.
There's also the problem of the orphan files: sometimes we have files that have been uploaded to the storage path, but haven't been committed as belonging to the table (due to interrupted transactions, crashed jobs, etc.). Since the Iceberg layer will only delete the files which it knows about, the orphan files will remain stored even after the table is deleted.
Adding a manual step
At this point, the steps to properly delete an Iceberg table (and all its associated elements) are:
- use the SQL interface to delete the table;
- use the storage layer's tools to delete the rest of the table.
Surely it cannot get worse, right?
Well, it gets worse: if you swap the order of the steps above, the table enters an unrecoverable state.
More precisely, you cannot remove the table from the catalog anymore,
if you start by deleting its files, without doing the DROP TABLE command beforehand
(and the table-name remains allocated forever). It's another known issue
(issue 1, issue 2, issue 3, issue 4) with proposed fixes (proposed fix 1, proposed fix 2) but no merged solution.
And then there's Hadoop
When using a Hadoop catalog, a simple DROP TABLE is sufficient to delete everything: files and directories.
It's strange that just by changing the catalog type (which is just a detail related to the
metadata management), the behavior of DROP TABLE changes completely, but it happens anyway.
import $ivy.`org.apache.spark::spark-core:4.0.2`
import $ivy.`org.apache.spark::spark-sql:4.0.2`
import $ivy.`org.apache.iceberg::iceberg-spark-runtime-4.0:1.10.1`
// Cleanup command:
// rm -rf database.db spark-warehouse/ warehouse/
import org.apache.spark.sql.SparkSession
@main
def main() : Unit = {
val spark = SparkSession
.builder()
.appName("Iceberg table deletion")
.master("local[*]")
.config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
.config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog")
.config("spark.sql.catalog.spark_catalog.type", "hadoop")
.config("spark.sql.catalog.spark_catalog.warehouse", "warehouse")
.getOrCreate()
spark.sql("""
CREATE TABLE db.table
USING iceberg
PARTITIONED BY (bucket(16, id))
AS (
VALUES (22, 'aa'), (33, 'bb'), (44, 'cc')
AS sample_val (id, data)
)
""")
spark.sql("DROP TABLE db.table")
}
Result:
.
|-- spark-warehouse
|-- test.sc
`-- warehouse
`-- db
Conclusion
Just because a table "has disappeared" from the SQL interface, don't assume that the files and directories themselves have been deleted. Do a manual check, since there might be leftovers.