Skip to content
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

Parse partition values before handing to PyArrow #565

Merged
merged 4 commits into from
Mar 10, 2022
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
1 change: 1 addition & 0 deletions Cargo.lock

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

2 changes: 1 addition & 1 deletion python/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -29,4 +29,4 @@ features = ["extension-module", "abi3", "abi3-py36"]
[dependencies.deltalake]
path = "../rust"
version = "0"
features = ["s3", "azure", "glue", "gcs"]
features = ["s3", "azure", "glue", "gcs", "python"]
4 changes: 3 additions & 1 deletion python/deltalake/table.py
Original file line number Diff line number Diff line change
Expand Up @@ -281,7 +281,9 @@ def to_pyarrow_dataset(
filesystem=filesystem,
partition_expression=part_expression,
)
for file, part_expression in self._table.dataset_partitions(partitions)
for file, part_expression in self._table.dataset_partitions(
partitions, self.pyarrow_schema()
)
]

return FileSystemDataset(fragments, self.pyarrow_schema(), format, filesystem)
Expand Down
17 changes: 15 additions & 2 deletions python/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -248,6 +248,7 @@ impl RawDeltaTable {
&mut self,
py: Python<'py>,
partition_filters: Option<Vec<(&str, &str, PartitionFilterValue)>>,
schema: ArrowSchema,
) -> PyResult<Vec<(String, Option<&'py PyAny>)>> {
let path_set = match partition_filters {
Some(filters) => Some(HashSet::<_>::from_iter(
Expand All @@ -266,7 +267,7 @@ impl RawDeltaTable {
})
.map(|((path, partition_values), stats)| {
let stats = stats.map_err(PyDeltaTableError::from_raw)?;
let expression = filestats_to_expression(py, partition_values, stats)?;
let expression = filestats_to_expression(py, &schema, partition_values, stats)?;
Ok((path, expression))
})
.collect()
Expand Down Expand Up @@ -297,19 +298,31 @@ fn json_value_to_py(value: &serde_json::Value, py: Python) -> PyObject {
/// skipped during a scan.
fn filestats_to_expression<'py>(
py: Python<'py>,
schema: &ArrowSchema,
partitions_values: &HashMap<String, Option<String>>,
stats: Option<Stats>,
) -> PyResult<Option<&'py PyAny>> {
let ds = PyModule::import(py, "pyarrow.dataset")?;
let field = ds.getattr("field")?;
let pa = PyModule::import(py, "pyarrow")?;
let mut expressions: Vec<PyResult<&PyAny>> = Vec::new();

for (column, value) in partitions_values.iter() {
if let Some(value) = value {
// value is a string, but needs to be parsed into appropriate type
let column_type = schema
.field_with_name(column)
.map_err(|_| PyDeltaTableError::new_err("Partition column not found in schema"))?
.data_type()
.clone()
.into_py(py);
let converted_value = pa
.call_method1("scalar", (value,))?
.call_method1("cast", (column_type,))?;
expressions.push(
field
.call1((column,))?
.call_method1("__eq__", (value.to_object(py),)),
.call_method1("__eq__", (converted_value,)),
);
}
}
Expand Down
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}
{"metaData":{"id":"588135b2-b298-4d9f-aab6-6dd9bf90d575","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["date"],"configuration":{},"createdTime":1645893400586}}
{"add":{"path":"date=2021-01-01/part-00000-6ae76612-3903-4007-b5ec-511b932751ea.c000.snappy.parquet","partitionValues":{"date":"2021-01-01"},"size":500,"modificationTime":1645893404567,"dataChange":true}}
{"commitInfo":{"timestamp":1645893404671,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"date\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"5","numOutputBytes":"500"},"engineInfo":"Apache-Spark/3.2.1 Delta-Lake/1.1.0"}}
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}
{"metaData":{"id":"e79e0060-0670-46ed-9e93-9dff7ac96b07","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["date"],"configuration":{},"createdTime":1645893413372}}
{"add":{"path":"date=2021-01-01%2000%253A00%253A00/part-00000-6177a755-69ce-44ea-bbfa-0479c4c1e704.c000.snappy.parquet","partitionValues":{"date":"2021-01-01 00:00:00"},"size":500,"modificationTime":1645893413662,"dataChange":true}}
{"commitInfo":{"timestamp":1645893413668,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"date\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"5","numOutputBytes":"500"},"engineInfo":"Apache-Spark/3.2.1 Delta-Lake/1.1.0"}}
Binary file not shown.
Binary file not shown.
18 changes: 17 additions & 1 deletion python/tests/test_table_read.py
Original file line number Diff line number Diff line change
@@ -1,12 +1,14 @@
import os
from datetime import date
from threading import Barrier, Thread

import pandas as pd
import pyarrow as pa
import pyarrow.dataset as ds
import pytest
from pyarrow.fs import LocalFileSystem

from deltalake import DeltaTable, Metadata
from deltalake import DeltaTable


def test_read_simple_table_to_dict():
Expand Down Expand Up @@ -93,6 +95,20 @@ def test_read_partitioned_table_to_dict():
assert dt.to_pyarrow_dataset().to_table().to_pydict() == expected


@pytest.mark.parametrize(
"name,partition_type", [("date", pa.date32()), ("timestamp", pa.timestamp("us"))]
)
def test_read_date_partitioned_table(name, partition_type):
table_path = f"tests/data/{name}_partitioned_df"
dt = DeltaTable(table_path)
table = dt.to_pyarrow_table()
assert table["date"].type == partition_type
date_expected = pa.array([date(2021, 1, 1)] * 5).cast(partition_type)
assert table["date"] == pa.chunked_array([date_expected])
id_expected = pa.array(range(5))
assert table["id"] == pa.chunked_array([id_expected])


def test_read_partitioned_table_with_partitions_filters_to_dict():
table_path = "../rust/tests/data/delta-0.8.0-partitioned"
dt = DeltaTable(table_path)
Expand Down
1 change: 1 addition & 0 deletions rust/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@ s3 = ["rusoto_core/native-tls", "rusoto_credential", "rusoto_s3/native-tls", "ru
s3-rustls = ["rusoto_core/rustls", "rusoto_credential", "rusoto_s3/rustls", "rusoto_sts/rustls", "rusoto_dynamodb/rustls", "maplit", "dynamodb_lock/rustls"]
gcs = ["async-stream", "tame-gcs", "tame-oauth", "reqwest"]
glue = ["s3", "rusoto_glue"]
python = ["arrow/pyarrow"]

[build-dependencies]
glibc_version = "0"
Expand Down