Skip to content

Schema evolution error when querying Parquet files with different schema versions #1529

@amitgilad3

Description

@amitgilad3

Apache Iceberg Rust version

0.5.1 (latest version)

Describe the bug

When querying an Iceberg table that has evolved its schema, iceberg-rust fails if some underlying Parquet files do not yet contain the newly added field(s). This violates the schema evolution contract in Iceberg, where missing fields should be treated as null. Instead, the reader returns a schema mismatch error.

Error: External(DataInvalid => Parquet schema table {
1: t_id: optional string
2: primal_country: optional string
3: other_countries: optional string
4: primary_country_iso2: optional string
5: other_countries_iso2: optional string
}
and Iceberg schema table {
1: t_id: optional string
2: primal_country: optional string
3: other_countries: optional string
4: primary_country_iso2: optional string
5: other_countries_iso2: optional string
6: event_time: optional timestamp
}
do not match.

i was looking through the slack and found this - #602 but im not sure this pr addresses this issue

To Reproduce

  1. spark.sql("""
    CREATE OR REPLACE TABLE my_namespace.demo (
    t_id BIGINT,
    primal_country STRING,
    other_countries STRING,
    primary_country_iso2 STRING,
    other_countries_iso2 STRING,
    event_time TIMESTAMP
    )
    USING iceberg
    """);

  2. spark.sql("""
    INSERT INTO demo (t_id, primal_country, other_countries, primary_country_iso2, other_countries_iso2)
    VALUES
    (1, 'USA', 'Canada,Mexico', 'US', 'CA,MX'),
    (2, 'Germany', 'France,Italy', 'DE', 'FR,IT')
    """)

  3. Now modify the table schema
    spark.sql("""ALTER TABLE demo
    ADD COLUMNS (
    event_time timestamp
    )""")

  4. Insert rows WITH event_time (evolved schema)
    spark.sql("""
    INSERT INTO demo (t_id, primal_country, other_countries, primary_country_iso2, other_countries_iso2, event_time)
    VALUES
    (3, 'India', 'Nepal,Bangladesh', 'IN', 'NP,BD', current_timestamp()),
    (4, 'UK', 'Ireland,Scotland', 'GB', 'IE,SCT', current_timestamp())
    """)

and now query the table using iceberg-rust - in my case i am working with aws-glue so i dont have a rust example to showcase

Expected behavior

The query should succeed, returning results from both the old and new Parquet files. Files that don't contain the newly added ingest_ts field should have null or default values for it. This behavior is consistent with how schema evolution is handled in Iceberg (and in other implementations like Java or Python).

Willingness to contribute

I would be willing to contribute a fix for this bug with guidance from the Iceberg community

Metadata

Metadata

Assignees

No one assigned

    Labels

    bugSomething isn't working

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions