Skip to content

Commit

Permalink
Parse partition values before handing to PyArrow
Browse files Browse the repository at this point in the history
We were providing all partition values as strings, by PyArrow expects
them to be parsed to the correct type.
  • Loading branch information
wjones127 committed Feb 26, 2022
1 parent f5e4b5f commit fd7e0a9
Show file tree
Hide file tree
Showing 14 changed files with 74 additions and 5 deletions.
2 changes: 2 additions & 0 deletions Cargo.lock

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

1 change: 1 addition & 0 deletions python/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ env_logger = "0"
reqwest = { version = "*", features = ["native-tls-vendored"] }
serde_json = "1"
chrono = "0"
arrow = { version = "9", features = ["pyarrow"] }

[dependencies.pyo3]
version = "0.15"
Expand Down
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
20 changes: 17 additions & 3 deletions python/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@

extern crate pyo3;

use arrow::pyarrow::PyArrowConvert;
use chrono::{DateTime, FixedOffset, Utc};
use deltalake::action::Stats;
use deltalake::action::{ColumnCountStat, ColumnValueStat};
Expand All @@ -10,7 +11,7 @@ use deltalake::partitions::PartitionFilter;
use deltalake::storage;
use deltalake::{arrow, StorageBackend};
use pyo3::create_exception;
use pyo3::exceptions::PyException;
use pyo3::exceptions::{PyAssertionError, PyException};
use pyo3::prelude::*;
use pyo3::types::{PyBytes, PyTuple, PyType};
use std::collections::HashMap;
Expand Down Expand Up @@ -248,6 +249,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 +268,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 +299,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 {
// TODO: value is a string, but needs to be parsed into appropriate type
let column_type = schema
.field_with_name(column)
.map_err(|_| PyAssertionError::new_err("Partition column not found in schema"))?
.data_type()
.to_pyarrow(py)?;
// pa.scalar(value).cast(type)
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.
26 changes: 26 additions & 0 deletions python/tests/data/gen_date_data.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
"""Generate test data."""
# TODO: Once we have a writer, replace this script.
from delta import configure_spark_with_delta_pip
from pyspark.sql import SparkSession
from pyspark.sql import functions as F

builder = (
SparkSession.builder.appName("MyApp")
.config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
.config(
"spark.sql.catalog.spark_catalog",
"org.apache.spark.sql.delta.catalog.DeltaCatalog",
)
)

spark = configure_spark_with_delta_pip(builder).getOrCreate()

df = spark.range(5).repartition(1).withColumn("date", F.lit("2021-01-01").cast("date"))
df.write.partitionBy("date").format("delta").save("date_partitioned_df")

df = (
spark.range(5)
.repartition(1)
.withColumn("date", F.lit("2021-01-01").cast("timestamp"))
)
df.write.partitionBy("date").format("delta").save("timestamp_partitioned_df")
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

0 comments on commit fd7e0a9

Please sign in to comment.