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

Error with MERGE for partitioned table having delta.enableChangeDataFeed set to true and non-primitive columns #2832

Closed
ximonsson opened this issue Aug 27, 2024 · 3 comments · Fixed by #2919
Assignees
Labels
binding/rust Issues for the Rust crate bug Something isn't working

Comments

@ximonsson
Copy link

Environment

Delta-rs version: 0.19.1

Binding: python

Environment:

  • Cloud provider:
  • OS:
  • Other:

Bug

What happened:

This error seems to have been introduced in 0.19. An error is thrown when trying to merge into a table that is partitioned and has the delta.enableChangeDataFeed configuration set to true, and if any of the columns have a non-primitive type, e.g. timestamp or array.

What you expected to happen:

Would be niced if there was no error I guess.

How to reproduce it:

import deltalake
import duckdb

# create dataset

duckdb.sql(
    "CREATE TABLE cowabunga (magic INTEGER, name VARCHAR, bday TIMESTAMP, weapons VARCHAR[])"
)
duckdb.sql(
    """
    INSERT INTO cowabunga VALUES
        (1, 'Leonardo', TIMESTAMP '2024-01-01 00:00:00', ['ninjato']),
        (1, 'Donatello', TIMESTAMP '2024-01-01 01:00:00', ['longstick']),
        (2, 'Michelangelo', TIMESTAMP '2024-01-01 02:00:00', ['nunchucks']),
        (2, 'Raphael', TIMESTAMP '2024-01-01 03:00:00', ['forks - whatever they are called']),
"""
)

# write table for first time

print("create table")

data = duckdb.sql("SELECT * FROM cowabunga LIMIT 2").fetch_arrow_reader()
deltalake.write_deltalake(
    "/tmp/tmnt",
    data,
    mode="overwrite",
    partition_by=["magic"],
    configuration={"delta.enableChangeDataFeed": "true"},
)

# merge rest of data

print("merge rest of data")

data = duckdb.table("cowabunga").fetch_arrow_reader()
(
    deltalake.DeltaTable("/tmp/tmnt")
    .merge(source=data, target_alias="t", source_alias="s", predicate="s.name = t.name")
    .when_matched_update_all()
    .when_not_matched_insert_all()
    .execute()
)

This produces the following error:

_internal.DeltaError: Generic DeltaTable error: Error during planning: UNION Column bday (type: Timestamp(Microsecond, None)) is not compatible with column weapons (type: List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }))

Removing either partition_by in the first write, the delta.enableChangeDataFeed or the columns bday and weapons fixes it.

More details:

I tried using version 0.18 and there it works.

@ximonsson ximonsson added the bug Something isn't working label Aug 27, 2024
@rtyler rtyler added the binding/rust Issues for the Rust crate label Sep 3, 2024
@rtyler
Copy link
Member

rtyler commented Sep 3, 2024

This looks like a bug in our change data feed handling. Prior to 0.19.0 the Rust crate did not support CDF on the merge operation, which will be triggered by the enableChangeDataFeed table property.

😞

@rtyler rtyler self-assigned this Sep 3, 2024
@sherlockbeard
Copy link
Contributor

This looks like a bug in our change data feed handling

change_data.push(tracker.collect()?);

it's dying here but this function is straight forward except logic 😿

@VillePuuska
Copy link
Contributor

If the partition column is the last column this issue doesn't seem to pop up. With some debug printing it looks like the schemas of preimage and postimage have the partition column in a different spot here

let final_df = preimage.union(postimage)?;

For example, adding

println!("{}", preimage.schema());
println!("{}", postimage.schema());

before this line and running the above Python script prints

fields:[t.name, t.bday, t.weapons, t.magic, _change_type], metadata:{}
fields:[magic, name, bday, weapons, _change_type], metadata:{}

before the error.

Not sure why the union would even go through with ints and strings getting mixed, but not if there are timestamps or arrays in the mix 🤔 Ofc I could just be completely off base and this column moving isn't even related to the issue.

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

Successfully merging a pull request may close this issue.

5 participants