ray.data.read_unity_catalog#
- ray.data.read_unity_catalog(table: str, url: str, token: str, *, data_format: str | None = None, region: str | None = None, reader_kwargs: dict | None) Dataset [source]#
Loads a Unity Catalog table or files into a Ray Dataset using Databricks Unity Catalog credential vending, with automatic short-lived cloud credential handoff for secure, parallel, distributed access from external engines.
This function works by leveraging Unity Catalog’s credential vending feature, which grants temporary, least-privilege credentials for the cloud storage location backing the requested table or data files. It authenticates via the Unity Catalog REST API (
Unity Catalog credential vending for external system access
, [Databricks Docs](https://docs.databricks.com/en/data-governance/unity-catalog/credential-vending.html)), ensuring that permissions are enforced at the Databricks principal (user, group, or service principal) making the request. The function supports reading data directly from AWS S3, Azure Data Lake, or GCP GCS in standard formats including Delta and Parquet.Note
This
read_unity_catalog
function is currently experimental and under active developmentWarning
The Databricks Unity Catalog credential vending feature is currently in Public Preview and there are important requirements and limitations. You must read these docs carefully and ensure your workspace and principal are properly configured.
- Features:
Secure Access: Only principals with
EXTERNAL USE SCHEMA
on the containing schema, and after explicit metastore enablement, can obtain short-lived credentials.Format Support: Supports reading
delta
andparquet
formats via supported Ray Dataset readers (iceberg coming soon).Cloud Support: AWS, Azure, and GCP supported, with automatic environment setup for the vended credentials per session.
Auto-Infer: Data format is auto-inferred from table metadata, but can be explicitly specified.
Examples
Read a Unity Catalog managed Delta table with credential vending:
>>> import ray >>> ds = read_unity_catalog( ... table="main.sales.transactions", ... url="https://dbc-XXXXXXX-XXXX.cloud.databricks.com", # noqa: E501 ... token="XXXXXXXXXXX" # noqa: E501 ... ) >>> ds.show(3)
Explicitly specify the format, and pass reader options:
>>> ds = read_unity_catalog( ... table="main.catalog.images", ... url="https://dbc-XXXXXXX-XXXX.cloud.databricks.com", # noqa: E501 ... token="XXXXXXXXXXX", # noqa: E501 ... data_format="delta", ... region="us-west-2", ... # Reader kwargs come from the associated reader (ray.data.read_delta in this example) ... reader_kwargs={"override_num_blocks": 1000} ... )
- Parameters:
table – Unity Catalog table name as
<catalog>.<schema>.<table>
. Must be a managed or external table supporting credential vending.url – Databricks workspace URL, e.g.
"https://dbc-XXXXXXX-XXXX.cloud.databricks.com"
token – Databricks PAT (Personal Access Token) with
EXTERNAL USE SCHEMA
on the schema containing the table, and with access to the workspace API.data_format – (Optional) Data format override. If not specified, inferred from Unity Catalog metadata and file extension. Supported:
"delta"
,"parquet"
region – (Optional) For S3: AWS region for cloud credential environment setup.
reader_kwargs – Additional arguments forwarded to the underlying Ray Dataset reader (e.g., override_num_blocks, etc.).
- Returns:
A
ray.data.Dataset
containing the data from the external Unity Catalog table.
References
Databricks Credential Vending: https://docs.databricks.com/en/data-governance/unity-catalog/credential-vending.html
API Reference for temporary credentials: https://docs.databricks.com/api/workspace/unity-catalog/temporary-table-credentials
PublicAPI (alpha): This API is in alpha and may change before becoming stable.