Skip to content
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
4 changes: 2 additions & 2 deletions python/Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

install:
pip install poetry
poetry install -E pyarrow -E hive -E s3fs -E glue -E adlfs -E duckdb
poetry install -E pyarrow -E hive -E s3fs -E glue -E adlfs -E duckdb -E ray

check-license:
./dev/check-license
Expand All @@ -36,7 +36,7 @@ test-integration:
docker-compose -f dev/docker-compose-integration.yml kill
docker-compose -f dev/docker-compose-integration.yml build
docker-compose -f dev/docker-compose-integration.yml up -d
sleep 20
sleep 30
Copy link
Contributor Author

@JonasJ-ap JonasJ-ap Mar 23, 2023

Choose a reason for hiding this comment

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

According to #7163 (comment), #7163 (comment), it seems sometimes the iceberg-spark docker need more time to finish table creation. Extending the wait time to 30s to ensure the integration test can pass.

poetry run pytest tests/ -m integration ${PYTEST_ARGS}

test-adlfs:
Expand Down
30 changes: 30 additions & 0 deletions python/dev/provision.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import time

from pyspark.sql import SparkSession
from pyspark.sql.functions import current_date, date_add, expr

spark = SparkSession.builder.getOrCreate()

Expand Down Expand Up @@ -56,6 +57,12 @@
"""
)

spark.sql(
"""
DROP TABLE IF EXISTS test_null_nan_rewritten;
"""
)

spark.sql(
"""
CREATE TABLE test_null_nan_rewritten
Expand Down Expand Up @@ -94,5 +101,28 @@
"""
)

all_types_dataframe = (
spark.range(0, 5, 1, 5)
.withColumnRenamed("id", "longCol")
.withColumn("intCol", expr("CAST(longCol AS INT)"))
.withColumn("floatCol", expr("CAST(longCol AS FLOAT)"))
.withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)"))
.withColumn("dateCol", date_add(current_date(), 1))
.withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)"))
.withColumn("stringCol", expr("CAST(dateCol AS STRING)"))
.withColumn("booleanCol", expr("longCol > 5"))
.withColumn("binaryCol", expr("CAST(longCol AS BINARY)"))
.withColumn("byteCol", expr("CAST(longCol AS BYTE)"))
.withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(10, 2))"))
.withColumn("shortCol", expr("CAST(longCol AS SHORT)"))
.withColumn("mapCol", expr("MAP(longCol, decimalCol)"))
.withColumn("arrayCol", expr("ARRAY(longCol)"))
.withColumn("structCol", expr("STRUCT(mapCol, arrayCol)"))
)

all_types_dataframe.writeTo("default.test_all_types").tableProperty("format-version", "2").partitionedBy(
"intCol"
).createOrReplace()

while True:
time.sleep(1)
48 changes: 48 additions & 0 deletions python/mkdocs/docs/api.md
Original file line number Diff line number Diff line change
Expand Up @@ -369,3 +369,51 @@ print(
(datetime.timedelta(seconds=1581),),
]
```

### Ray

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

A table scan can also be converted into a Ray dataset:

```python
ray_dataset = table.scan(
row_filter=GreaterThanOrEqual("trip_distance", 10.0),
selected_fields=("VendorID", "tpep_pickup_datetime", "tpep_dropoff_datetime"),
).to_ray()
```

This will return a Ray dataset:

```
Dataset(
num_blocks=1,
num_rows=1168798,
schema={
VendorID: int64,
tpep_pickup_datetime: timestamp[us, tz=UTC],
tpep_dropoff_datetime: timestamp[us, tz=UTC]
}
)
```

Using [Ray Dataset API](https://docs.ray.io/en/latest/data/api/dataset.html) to interact with the dataset:

```python
print(
ray_dataset.take(2)
)
[
{
'VendorID': 2,
'tpep_pickup_datetime': datetime.datetime(2008, 12, 31, 23, 23, 50, tzinfo=<UTC>),
'tpep_dropoff_datetime': datetime.datetime(2009, 1, 1, 0, 34, 31, tzinfo=<UTC>)
},
{
'VendorID': 2,
'tpep_pickup_datetime': datetime.datetime(2008, 12, 31, 23, 5, 3, tzinfo=<UTC>),
'tpep_dropoff_datetime': datetime.datetime(2009, 1, 1, 16, 10, 18, tzinfo=<UTC>)
}
]
```
2 changes: 2 additions & 0 deletions python/mkdocs/docs/index.md
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,9 @@ You can mix and match optional dependencies depending on your needs:
| glue | Support for AWS Glue |
| dynamodb | Support for AWS DynamoDB |
| pyarrow | PyArrow as a FileIO implementation to interact with the object store |
| pandas | Installs both PyArrow and Pandas |
| duckdb | Installs both PyArrow and DuckDB |
| ray | Installs PyArrow, Pandas, and Ray |
| s3fs | S3FS as a FileIO implementation to interact with the object store |
| adlfs | ADLFS as a FileIO implementation to interact with the object store |
| snappy | Support for snappy Avro compression |
Expand Down
393 changes: 385 additions & 8 deletions python/poetry.lock

Large diffs are not rendered by default.

6 changes: 6 additions & 0 deletions python/pyiceberg/table/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@
if TYPE_CHECKING:
import pandas as pd
import pyarrow as pa
import ray
from duckdb import DuckDBPyConnection


Expand Down Expand Up @@ -415,3 +416,8 @@ def to_duckdb(self, table_name: str, connection: Optional[DuckDBPyConnection] =
con.register(table_name, self.to_arrow())

return con

def to_ray(self) -> ray.data.dataset.Dataset:
import ray

return ray.data.from_arrow(self.to_arrow())
Copy link

Choose a reason for hiding this comment

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

just for my understanding, would self.to_arrow() returns a pa.Table has all rows from Iceberg table? So that would assume the table is very small to fit in the memory of Python client, right? What if the table is larger than the available memory?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thank you for mentioning this. to_arrow() will convert all the rows and fields(columns) that are included in the current table scan to a pa.Table. Row filters or field selectors could be applied to the table scan to exclude unwanted rows or fields. e.g.

 not_nan = table_test_null_nan.scan(row_filter=NotNaN("col_numeric"), selected_fields=("idx",)).to_arrow()

The huge table size will be an issue for to_arrow if the table scan ends up including the whole table (or too many rows). There is an open PR #7163 that may help in this case by adding a limit to the number of rows included in the table scan. Here is one example quoted from that PR:

limited_result = table_test_limit.scan(selected_fields=("idx",), limit=20).to_arrow()

In the future, I think there could be more discussions on converting large iceberg table to ray dataset. Please let me know if you have more questions/concerns on this.

Copy link
Contributor

Choose a reason for hiding this comment

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

Is this the same experience today for connections like duckdb?

Copy link
Contributor Author

@JonasJ-ap JonasJ-ap Mar 22, 2023

Choose a reason for hiding this comment

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

I think so, since currently all the connections are based on to_arrow(). I draw the above conclusion based on the discussion here: #6505 (comment), #6505 (comment)

An out-of-memory error occurs when converting a large table to duckdb on an EC2 instance with insufficient memory.

Copy link
Contributor

Choose a reason for hiding this comment

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

Got it. I guess we need to address this issue across connections then. Thanks for the explanation!

Copy link
Contributor

Choose a reason for hiding this comment

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

Arrow also allows to_batches which will consume the source in batches, but not sure if Ray can leverage this. In the end, we want to leverage the PyArrow dataset, but this requires much deeper integration with Arrow. I'm digging into this, but no design yet. Maybe we can send a substrait plan? Open for discussion

Copy link

Choose a reason for hiding this comment

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

It looks like self.to_arrow() will eventually call pq.read_table() to read the table into memory (with optionally row filtering and columns pruning) (to_arrow -> project_table -> _file_to_table -> pq.read_table).

Ray also leverages PyArrow dataset to read a variety of format (Parquet, CSV, etc) (API list). And internally Ray uses PyArrow to read data in batches and in parallel with many tasks.

This PR is a good starting point. We can further collaborate to add a read_iceberg() API in Ray, to implement a custom data source for Iceberg, so it can do the parallel read and be scalable.

7 changes: 7 additions & 0 deletions python/pyproject.toml
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,8 @@ pandas = { version = ">=1.4.4,<=1.5.3", optional = true }

duckdb = { version = ">=0.6.0,<=0.7.1", optional = true }

ray = { version = ">=2.0.0,<=2.3.0", optional = true }

python-snappy = { version = "0.6.1", optional = true }

thrift = { version = "0.16.0", optional = true }
Expand Down Expand Up @@ -100,6 +102,7 @@ build-backend = "poetry.core.masonry.api"
pyarrow = ["pyarrow"]
pandas = ["pandas", "pyarrow"]
duckdb = ["duckdb", "pyarrow"]
ray = ["ray", "pyarrow", "pandas"]
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we need pandas as well?

Copy link

Choose a reason for hiding this comment

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

Sorry wondering what's this pyproject.toml used for? Is it the required dependency?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@Fokko Yes. The pandas seems to be a required dependency when converting from pa.Table to ray dataset.

Here is the trace when running table.scan().to_ray() without pandas:

2023-03-22 19:22:32,946	INFO worker.py:772 -- Task failed with retryable exception: TaskID(c8ef45ccd0112571ffffffffffffffffffffffff01000000).
(_get_metadata pid=77891) Traceback (most recent call last):
(_get_metadata pid=77891)   File "python/ray/_raylet.pyx", line 857, in ray._raylet.execute_task
(_get_metadata pid=77891)   File "python/ray/_raylet.pyx", line 861, in ray._raylet.execute_task
(_get_metadata pid=77891)   File "/Users/jonasjiang/Library/Caches/pypoetry/virtualenvs/pyiceberg-oQQETec1-py3.9/lib/python3.9/site-packages/ray/data/read_api.py", line 1565, in _get_metadata
(_get_metadata pid=77891)     return BlockAccessor.for_block(table).get_metadata(
(_get_metadata pid=77891)   File "/Users/jonasjiang/Library/Caches/pypoetry/virtualenvs/pyiceberg-oQQETec1-py3.9/lib/python3.9/site-packages/ray/data/block.py", line 379, in for_block
(_get_metadata pid=77891)     import pandas
(_get_metadata pid=77891) ModuleNotFoundError: No module named 'pandas'

Here is the reference for for_block method in ray:
https://github.com/ray-project/ray/blob/2e6de87ada84609fcd4f85bd16b9c2e090e921ae/python/ray/data/block.py#L379-L394

The import pandas cause this requirement.

Copy link
Contributor Author

@JonasJ-ap JonasJ-ap Mar 22, 2023

Choose a reason for hiding this comment

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

@c21 The pyproject.toml tracks both the required and optional dependencies. Currently, ray is set as an optional dependency

ray = { version = ">=2.0.0,<=2.3.0", optional = true }

For example. If users want to convert the iceberg table stored in AWS Glue to a ray dataset, they can install the pyiceberg by

git clone https://github.com/apache/iceberg.git
cd iceberg/python
pip3 install -e ".[glue,ray]"

In this way, all the required dependencies for ray are installed, including ray, pyarrow, and pandas as specified in line 105

ref: https://py.iceberg.apache.org/

Copy link

Choose a reason for hiding this comment

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

If this file represents required dependency, this change looks good to me.

snappy = ["python-snappy"]
hive = ["thrift"]
s3fs = ["s3fs"]
Expand Down Expand Up @@ -237,6 +240,10 @@ ignore_missing_imports = true
module = "duckdb.*"
ignore_missing_imports = true

[[tool.mypy.overrides]]
module = "ray.*"
ignore_missing_imports = true

[[tool.mypy.overrides]]
module = "pyparsing.*"
ignore_missing_imports = true
Expand Down
38 changes: 38 additions & 0 deletions python/tests/test_integration.py
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,11 @@ def table_test_null_nan_rewritten(catalog: Catalog) -> Table:
return catalog.load_table("default.test_null_nan_rewritten")


@pytest.fixture()
def table_test_all_types(catalog: Catalog) -> Table:
return catalog.load_table("default.test_all_types")


@pytest.mark.integration
def test_pyarrow_nan(table_test_null_nan: Table) -> None:
arrow_table = table_test_null_nan.scan(row_filter=IsNaN("col_numeric"), selected_fields=("idx", "col_numeric")).to_arrow()
Expand Down Expand Up @@ -80,3 +85,36 @@ def test_duckdb_nan(table_test_null_nan_rewritten: Table) -> None:
result = con.query("SELECT idx, col_numeric FROM table_test_null_nan WHERE isnan(col_numeric)").fetchone()
assert result[0] == 1
assert math.isnan(result[1])


@pytest.mark.integration
def test_ray_nan(table_test_null_nan_rewritten: Table) -> None:
ray_dataset = table_test_null_nan_rewritten.scan().to_ray()
assert ray_dataset.count() == 3
assert math.isnan(ray_dataset.take()[0]["col_numeric"])


@pytest.mark.integration
def test_ray_nan_rewritten(table_test_null_nan_rewritten: Table) -> None:
ray_dataset = table_test_null_nan_rewritten.scan(
row_filter=IsNaN("col_numeric"), selected_fields=("idx", "col_numeric")
).to_ray()
assert ray_dataset.count() == 1
assert ray_dataset.take()[0]["idx"] == 1
assert math.isnan(ray_dataset.take()[0]["col_numeric"])


@pytest.mark.integration
@pytest.mark.skip(reason="Fixing issues with NaN's: https://github.com/apache/arrow/issues/34162")
def test_ray_not_nan_count(table_test_null_nan_rewritten: Table) -> None:
ray_dataset = table_test_null_nan_rewritten.scan(row_filter=NotNaN("col_numeric"), selected_fields=("idx",)).to_ray()
print(ray_dataset.take())
assert ray_dataset.count() == 2


@pytest.mark.integration
def test_ray_all_types(table_test_all_types: Table) -> None:
ray_dataset = table_test_all_types.scan().to_ray()
pandas_dataframe = table_test_all_types.scan().to_pandas()
assert ray_dataset.count() == pandas_dataframe.shape[0]
assert pandas_dataframe.equals(ray_dataset.to_pandas())