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

Zorder after schema evolution error - "Cannot cast file schema" #3185

Open
liamphmurphy opened this issue Feb 3, 2025 · 2 comments
Open
Labels
bug Something isn't working

Comments

@liamphmurphy
Copy link
Contributor

liamphmurphy commented Feb 3, 2025

Environment

Delta-rs version: 0.24.0 (goes back at least as far as v0.16.4)

Binding: python

Environment: Local


Bug

What happened:
When a partition has data written that is the result of a schema_mode=merge call with a schema change, a z_order optimization fails with this error:

_internal.DeltaError: Failed to parse parquet: Parquet error: Z-order failed while scanning data: Plan("Cannot cast file schema field details of type Struct([Field { name: \"email\", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }]) to table schema field of type Struct([Field { name: \"email\", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: \"address\", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }])")

NOTE: MRE is below. This seems to only happen when a field is added WITHIN a Pyarrow schema struct.

What you expected to happen:
Z_order optimization would work as expected despite a schema evolution.

How to reproduce it:

Here is an MRE:

import pandas as pd
import pyarrow as pa
from deltalake import write_deltalake, DeltaTable

# Define the path for the Delta table
delta_table_path = "./test-table"

# Define the data for the first write
data_first_write = [
    {"name": "Alice", "age": 30, "details": {"email": "[email protected]"}},
    {"name": "Bob", "age": 25, "details": {"email": "[email protected]"}}
]

# Define the data for the second write with an additional 'address' field
data_second_write = [
    {"name": "Charlie", "age": 35, "details": {"email": "[email protected]", "address": "123 Main St"}},
    {"name": "Diana", "age": 28, "details": {"email": "[email protected]", "address": "456 Elm St"}}
]

# Define Pyarrow schemas for the first and second writes
schema_first_write = pa.schema([
    ("name", pa.string()),
    ("age", pa.int32()),
    ("details", pa.struct([
        ("email", pa.string())
    ]))
])

schema_second_write = pa.schema([
    ("name", pa.string()),
    ("age", pa.int32()),
    ("details", pa.struct([
        ("email", pa.string()),
        ("address", pa.string())  # New field in the second schema
    ]))
])

# Convert data to DataFrames
df_first_write = pd.DataFrame(data_first_write)
df_second_write = pd.DataFrame(data_second_write)

# Convert DataFrames to Pyarrow Tables using the defined schemas
table_first_write = pa.Table.from_pandas(df_first_write, schema=schema_first_write)
table_second_write = pa.Table.from_pandas(df_second_write, schema=schema_second_write)

# Write data to Delta table for the first write
write_deltalake(delta_table_path, table_first_write, mode="append", engine="rust")

# Write data to Delta table for the second write
write_deltalake(delta_table_path, table_second_write, mode="append", engine="rust", schema_mode="merge")

print("Data written to Delta table successfully.")

# Optionally optimize the Delta table
DeltaTable(delta_table_path).optimize.z_order(columns=["name"])
print("Optimize successful")

More details:

There are two cases I've seen where this DOES NOT happen:

  • As stated above, this only happens if a field is added to a nested pa.struct (the address field in this test)
    • A column added to the top level schema (not nested) does not fail
  • Normal optimize calls (no z-order) DOES NOT fail, works as expected.
    DeltaTable(delta_table_path).optimize.compact()
@liamphmurphy liamphmurphy added the bug Something isn't working label Feb 3, 2025
@liamphmurphy
Copy link
Contributor Author

FWIW this seems similar to this issue: #1927

@ion-elgreco
Copy link
Collaborator

ion-elgreco commented Feb 10, 2025

@liamphmurphy this should be an easy fix. Instead of reading the files with read_parquet we should use the Deltatable provider with a scan config where we do "with_files". The DeltaScan uses a SchemaAdapterFactory to map the incoming batches.

It's quite doable, so could you make a fix for it?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

No branches or pull requests

2 participants