Skip to content

add iceberg datafusion integration #2075

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 4 commits into from
Jul 6, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
63 changes: 63 additions & 0 deletions mkdocs/docs/api.md
Original file line number Diff line number Diff line change
Expand Up @@ -1758,3 +1758,66 @@ shape: (11, 4)
│ 21 ┆ 566 ┆ Incorrect billing amount ┆ 2022-04-17 10:53:20 │
└───────────┴─────────────┴────────────────────────────┴─────────────────────┘
```

### Apache DataFusion

PyIceberg integrates with [Apache DataFusion](https://datafusion.apache.org/) through the Custom Table Provider interface ([FFI_TableProvider](https://datafusion.apache.org/python/user-guide/io/table_provider.html)) exposed through `iceberg-rust`.

<!-- prettier-ignore-start -->

!!! note "Requirements"
This requires [`datafusion` to be installed](index.md).

<!-- prettier-ignore-end -->

<!-- markdownlint-disable MD046 -- Allowing indented multi-line formatting in admonition-->

!!! warning "Experimental Feature"
The DataFusion integration is considered **experimental**.

The integration has a few caveats:

- Only works with `datafusion >= 45`
- Depends directly on `iceberg-rust` instead of PyIceberg's implementation
- Has limited features compared to the full PyIceberg API

The integration will improve as both DataFusion and `iceberg-rust` matures.

<!-- markdownlint-enable MD046 -->

PyIceberg tables can be registered directly with DataFusion's SessionContext using the table provider interface.

```python
from datafusion import SessionContext
from pyiceberg.catalog import load_catalog
import pyarrow as pa

# Load catalog and create/load a table
catalog = load_catalog("catalog", type="in-memory")
catalog.create_namespace_if_not_exists("default")

# Create some sample data
data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]})
iceberg_table = catalog.create_table("default.test", schema=data.schema)
iceberg_table.append(data)

# Register the table with DataFusion
ctx = SessionContext()
ctx.register_table_provider("test", iceberg_table)

# Query the table using DataFusion SQL
ctx.table("test").show()
```

This will output:

```python
DataFrame()
+---+---+
| x | y |
+---+---+
| 1 | 4 |
| 2 | 5 |
| 3 | 6 |
+---+---+
```
42 changes: 22 additions & 20 deletions poetry.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

46 changes: 46 additions & 0 deletions pyiceberg/table/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,7 @@
import pyarrow as pa
import ray
from duckdb import DuckDBPyConnection
from pyiceberg_core.datafusion import IcebergDataFusionTable

from pyiceberg.catalog import Catalog

Expand Down Expand Up @@ -1494,6 +1495,51 @@ def to_polars(self) -> pl.LazyFrame:

return pl.scan_iceberg(self)

def __datafusion_table_provider__(self) -> "IcebergDataFusionTable":
"""Return the DataFusion table provider PyCapsule interface.

To support DataFusion features such as push down filtering, this function will return a PyCapsule
interface that conforms to the FFI Table Provider required by DataFusion. From an end user perspective
you should not need to call this function directly. Instead you can use ``register_table_provider`` in
the DataFusion SessionContext.

Returns:
A PyCapsule DataFusion TableProvider interface.

Example:
```python
from datafusion import SessionContext
from pyiceberg.catalog import load_catalog
import pyarrow as pa
catalog = load_catalog("catalog", type="in-memory")
catalog.create_namespace_if_not_exists("default")
data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]})
iceberg_table = catalog.create_table("default.test", schema=data.schema)
iceberg_table.append(data)
ctx = SessionContext()
ctx.register_table_provider("test", iceberg_table)
ctx.table("test").show()
```
Results in
```
DataFrame()
+---+---+
| x | y |
+---+---+
| 1 | 4 |
| 2 | 5 |
| 3 | 6 |
+---+---+
```
"""
from pyiceberg_core.datafusion import IcebergDataFusionTable

return IcebergDataFusionTable(
identifier=self.name(),
metadata_location=self.metadata_location,
file_io_properties=self.io.properties,
).__datafusion_table_provider__()


class StaticTable(Table):
"""Load a table directly from a metadata file (i.e., without using a catalog)."""
Expand Down
3 changes: 2 additions & 1 deletion pyproject.toml
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@ pyiceberg-core = { version = "^0.5.1", optional = true }
polars = { version = "^1.21.0", optional = true }
thrift-sasl = { version = ">=0.4.3", optional = true }
kerberos = {version = "^1.3.1", optional = true}
datafusion = { version = ">=45", optional = true }

[tool.poetry.group.dev.dependencies]
pytest = "7.4.4"
Expand All @@ -99,7 +100,6 @@ pytest-mock = "3.14.1"
pyspark = "3.5.6"
cython = "3.1.2"
deptry = ">=0.14,<0.24"
datafusion = ">=44,<48"
docutils = "!=0.21.post1" # https://github.com/python-poetry/poetry/issues/9248#issuecomment-2026240520
mypy-boto3-glue = ">=1.28.18"
mypy-boto3-dynamodb = ">=1.28.18"
Expand Down Expand Up @@ -314,6 +314,7 @@ gcsfs = ["gcsfs"]
rest-sigv4 = ["boto3"]
hf = ["huggingface-hub"]
pyiceberg-core = ["pyiceberg-core"]
datafusion = ["datafusion"]

[tool.pytest.ini_options]
markers = [
Expand Down
64 changes: 64 additions & 0 deletions tests/table/test_datafusion.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.


from pathlib import Path

import pyarrow as pa
import pytest
from datafusion import SessionContext

from pyiceberg.catalog import Catalog, load_catalog


@pytest.fixture(scope="session")
def warehouse(tmp_path_factory: pytest.TempPathFactory) -> Path:
return tmp_path_factory.mktemp("warehouse")


@pytest.fixture(scope="session")
def catalog(warehouse: Path) -> Catalog:
catalog = load_catalog(
"default",
uri=f"sqlite:///{warehouse}/pyiceberg_catalog.db",
warehouse=f"file://{warehouse}",
)
return catalog


def test_datafusion_register_pyiceberg_table(catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
catalog.create_namespace_if_not_exists("default")
iceberg_table = catalog.create_table_if_not_exists(
"default.dataset",
schema=arrow_table_with_null.schema,
)
iceberg_table.append(arrow_table_with_null)

ctx = SessionContext()
ctx.register_table_provider("test", iceberg_table)

datafusion_table = ctx.table("test")
assert datafusion_table is not None

assert datafusion_table.to_arrow_table().to_pylist() == iceberg_table.scan().to_arrow().to_pylist()

from pandas.testing import assert_frame_equal

assert_frame_equal(
datafusion_table.to_arrow_table().to_pandas(),
iceberg_table.scan().to_arrow().to_pandas(),
)