Use UniForm to read Delta tables with Iceberg clients

Delta Lake Universal Format (UniForm) allows you to read Delta tables with Iceberg reader clients. This feature requires Databricks Runtime 14.3 LTS or above.

Important

For documentation for the legacy UniForm IcebergCompatV1 table feature, see Legacy UniForm IcebergCompatV1.

You can configure an external connection to have Unity Catalog act as an Iceberg catalog. See Read using the Unity Catalog Iceberg catalog endpoint.

UniForm Iceberg uses Zstandard instead of Snappy as the compression codec for underlying Parquet data files.

Note

UniForm metadata generation runs asynchronously on the compute used to write data to Delta tables, which might increase the driver resource usage.

How does UniForm work?

UniForm takes advantage of the fact that Delta Lake and Iceberg consist of Parquet data files and a metadata layer. UniForm automatically generates Iceberg metadata asynchronously, without rewriting data, so that Iceberg clients can read Delta tables. A single copy of the data files serves multiple formats.

Requirements

To enable UniForm Iceberg, the following requirements must be met:

Note

You cannot enable deletion vectors on a table with UniForm Iceberg enabled.

Use REORG to disable and purge deletion vectors while enabling UniForm Iceberg on an existing table with deletion vectors enabled. See Enable or upgrade using REORG.

Enable UniForm Iceberg

Important

Enabling Delta UniForm sets the Delta table feature IcebergCompatV2, a write protocol feature. Only clients that support this table feature can write to UniForm-enabled tables. You must use Databricks Runtime 14.3 LTS or above to write to Delta tables with this feature enabled.

You can turn off UniForm by unsetting the delta.universalFormat.enabledFormats table property. Upgrades to Delta Lake reader and writer protocol versions cannot be undone.

You must set the following table properties to enable UniForm Iceberg:

'delta.enableIcebergCompatV2' = 'true'
'delta.universalFormat.enabledFormats' = 'iceberg'

When you first enable UniForm, asynchronous metadata generation begins. This task must complete before external clients can query the table using Iceberg. See Check Iceberg metadata generation status.

For a list of limitations, see Limitations.

Enable during table creation

Column mapping must be enabled to use UniForm Iceberg. This happens automatically if you enable UniForm Iceberg during table creation, as in the following example:

CREATE TABLE T(c1 INT) TBLPROPERTIES(
  'delta.enableIcebergCompatV2' = 'true',
  'delta.universalFormat.enabledFormats' = 'iceberg');

Enable by altering an existing table

In Databricks Runtime 15.4 LTS and above, you can enable or upgrade UniForm Iceberg on an existing table using the following syntax:

ALTER TABLE table_name SET TBLPROPERTIES(
  'delta.enableIcebergCompatV2' = 'true',
  'delta.universalFormat.enabledFormats' = 'iceberg');

Enable or upgrade using REORG

You can use REORG to enable UniForm Iceberg and rewrite underlying data files, as in the following example:

REORG TABLE table_name APPLY (UPGRADE UNIFORM(ICEBERG_COMPAT_VERSION=2));

Use REORG if any of following are true:

  • Your table has deletion vectors enabled.
  • You previously enabled the IcebergCompatV1 version of UniForm Iceberg.
  • You need to read from Iceberg engines that don't support Hive-style Parquet files, such as Athena or Redshift.

When does UniForm generate Iceberg metadata?

Azure Databricks triggers metadata generation asynchronously after a Delta Lake write transaction completes. This metadata generation process uses the same compute that completed the Delta transaction.

Note

You can also manually trigger Iceberg metadata generation. See Manually trigger Iceberg metadata conversion.

To avoid write latencies associated with metadata generation, Delta tables with frequent commits might group multiple Delta commits into a single commit to Iceberg metadata.

Delta Lake ensures that only one metadata generation process is in progress at any time on a given compute resource. Commits that would trigger a second concurrent metadata generation process successfully commit to Delta, but don't trigger asynchronous Iceberg metadata generation. This prevents cascading latency for metadata generation for workloads with frequent commits (seconds to minutes between commits).

See Delta and Iceberg table versions.

Delta and Iceberg table versions

Delta Lake and Iceberg allow time travel queries using table versions or timestamps stored in table metadata.

In general, Delta table versions do not align to Iceberg versions by either the commit timestamp or the version ID. To verify which version of a Delta table a given version of an Iceberg table corresponds to, you can use the corresponding table properties. See Check Iceberg metadata generation status.

Check Iceberg metadata generation status

UniForm adds the following fields to Unity Catalog and Iceberg table metadata to track metadata generation status:

Metadata field Description
converted_delta_version The latest version of the Delta table for which Iceberg metadata was successfully generated.
converted_delta_timestamp The timestamp of the latest Delta commit for which Iceberg metadata was successfully generated.

On Azure Databricks, you can review these metadata fields by doing one of the following:

  • Reviewing the Delta Uniform Iceberg section returned by DESCRIBE EXTENDED table_name.
  • Reviewing table metadata with Catalog Explorer.
  • Using the REST API to get a table.

See documentation for your Iceberg reader client for how to review table properties outside Azure Databricks. For OSS Apache Spark, you can see these properties using the following syntax:

SHOW TBLPROPERTIES <table-name>;

Manually trigger Iceberg metadata conversion

You can manually trigger Iceberg metadata generation for the latest version of the Delta table. This operation runs synchronously, meaning that when it completes, the table contents available in Iceberg reflect the latest version of the Delta table available when the conversion process started.

This operation should not be necessary under normal conditions, but can help if you encounter the following:

  • A cluster terminates before automatic metadata generation succeeds.
  • An error or job failure interrupts metadata generation.
  • A client that does not support UniForm Iceberg metadata gneration writes to the Delta table.

Use the following syntax to manually trigger Iceberg metadata generation:

MSCK REPAIR TABLE <table-name> SYNC METADATA

See REPAIR TABLE.

Read using a metadata JSON path

Some Iceberg clients require that you provide a path to versioned metadata files to register external Iceberg tables. Each time UniForm converts a new version of the Delta table to Iceberg, it creates a new metadata JSON file.

Clients that use metadata JSON paths for configuring Iceberg include BigQuery. Refer to documentation for the Iceberg reader client for configuration details.

Delta Lake stores Iceberg metadata under the table directory, using the following pattern:

<table-path>/metadata/<version-number>-<uuid>.metadata.json

On Azure Databricks, you can review this metadata location by doing one of the following:

  • Reviewing the Delta Uniform Iceberg section returned by DESCRIBE EXTENDED table_name.
  • Reviewing table metadata with Catalog Explorer.
  • Using the following command with the REST API:
GET api/2.1/unity-catalog/tables/<catalog-name>.<schame-name>.<table-name>

The response includes the following information:

{
    ...
          "delta_uniform_iceberg": {
              "metadata_location":  "<cloud-storage-uri>/metadata/v<version-number>-<uuid>.metadata.json"
    }
}

Important

Path-based Iceberg reader clients might require manually updating and refreshing metadata JSON paths to read current table versions. Users might encounter errors when querying Iceberg tables using out-of-date versions as Parquet data files are removed from the Delta table with VACUUM.

Read using the Unity Catalog Iceberg catalog endpoint

Some Iceberg clients can connect to an Iceberg REST catalog. Unity Catalog provides a read-only implementation of the Iceberg REST catalog API for Delta tables with UniForm enabled using the endpoint /api/2.1/unity-catalog/iceberg. See the Iceberg REST API spec for details on using this REST API.

Clients known to support the Iceberg catalog API include Apache Spark, Flink, and Trino. Refer to documentation for the Iceberg reader client for configuration details.

Authentication and authorization

There are two requirements for accessing data registered in Unity Catalog using the api/2.1/unity-catalog/iceberg endpoint from external services:

  • Authenticate using OAuth or a Databricks personal access token. See Authenticate access to Azure Databricks resources.
  • Enable External data access for your metastore. This limits access to users who have the EXTERNAL USE SCHEMA privilege on the schema that the API is reading from.

Apache Spark configuration example

The following is an example of the settings used to configure OSS Apache Spark to read UniForm as Iceberg:

"spark.sql.extensions": "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions",
"spark.sql.catalog.unity": "org.apache.iceberg.spark.SparkCatalog",
"spark.sql.catalog.unity.type": "rest",
"spark.sql.catalog.unity.uri": "<api-root>/api/2.1/unity-catalog/iceberg",
"spark.sql.catalog.unity.token": "<your_personal_access_token>",
"spark.sql.catalog.unity.warehouse": "<uc_catalog_name>"

Substitute the full URL of the workspace in which you generated the personal access token for <api-root>.

When you query tables in Unity Catalog using Spark configurations, keep the following in mind:

  • Object identifiers use the pattern unity.<schema-name>.<table-name>.

    This pattern uses the same three-tier namespace that is used in Unity Catalog, but with the catalog name replaced by unity.

  • You need "spark.sql.extensions": "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions" only if you are running Iceberg-specific stored procedures.

  • If you are using a cloud provider for storage, you must add the cloud-specific Iceberg bundle jars as spark packages:

    • AWS: org.apache.iceberg:iceberg-aws-bundle:<iceberg-version>
    • Azure: org.apache.iceberg:iceberg-azure-bundle:<iceberg-version>
    • GCP: org.apache.iceberg:iceberg-gcp-bundle:<iceberg-version>

    For details, see the documentation for the Iceberg AWS integration for Spark.

REST API curl example

You can also use a REST API call like the one in this curl example to load a table:

curl -X GET -H "Authentication: Bearer $OAUTH_TOKEN" -H "Accept: application/json" \
https://<workspace-instance>/api/2.1/unity-catalog/iceberg/v1/catalogs/<uc_catalog_name>/namespaces/<uc_schema_name>/tables/<uc_table_name>

You should then receive a response like this:

{
  "metadata-location": "abfss://my-container@my-storage-account.dfs.core.chinacloudapi.cn/path/to/iceberg/table/metadata/file",
  "metadata": <iceberg-table-metadata-json>,
  "config": {
    "expires-at-ms": "<epoch-ts-in-millis>",
    "adls.sas-token.<storage-account-name>.dfs.core.chinacloudapi.cn": "<temporary-sas-token>"
  }
}

Note

The expires-at-ms field in the response indicates the expiration time of the credentials and has a default expiry time of one hour. For better performance, have the client cache the credentials until the expiration time before requesting a new one.

Limitations

The following limitations exist for all UniForm tables:

  • UniForm does not work on tables with deletion vectors enabled. See What are deletion vectors?.
  • Delta tables with UniForm enabled do not support VOID types.
  • Iceberg clients can only read from UniForm. Writes are not supported.
  • Iceberg reader clients might have individual limitations, regardless of UniForm. See documentation for your chosen client.
  • The recipients of Delta Sharing can only read the table as Delta, even when UniForm is enabled.
  • Some Delta Lake table features used by UniForm Iceberg are not supported by some Delta Sharing reader clients. See What is Delta Sharing?.

Change Data Feed works for Delta clients when UniForm is enabled, but does not have support in Iceberg.