aquwikimedia opened a new issue, #14043: URL: https://github.com/apache/iceberg/issues/14043
### Apache Iceberg version 1.9.2 (latest release) ### Query engine Spark ### Please describe the bug 🐞 At Wikimedia, we are increasingly adopting Iceberg in our data lake, but we’ve encountered a blocker that looks like a schema evolution issue when adding columns to structs nested within map fields (using Parquet). I’ve put together a minimal reproducible example with Docker here: https://gitlab.wikimedia.org/aqu/iceberg_nested_map_test Tested environments: • Spark 3.3.2, Iceberg 1.6.1, Parquet 1.12.2, Java 17 • Spark 3.5.6, Iceberg 1.9.2, Parquet 1.13.1, Java 17 Description: When I add a new column to a struct nested inside a map field, some subsequent SELECT queries fail with a field ID mismatch error. For example: -- 1. Create table with map containing struct ```sql CREATE TABLE test_map_struct_table ( page_id BIGINT, revision_content_slots MAP<STRING, STRUCT< content_body: STRING, content_format: STRING, content_model: STRING, content_sha1: STRING, content_size: BIGINT >>, revision_content_is_visible BOOLEAN, wiki_id STRING ) USING iceberg; ``` -- 2. Insert test data ```sql INSERT INTO test_map_struct_table VALUES (12345, MAP('main', NAMED_STRUCT( 'content_body', 'This is the main content of the page', 'content_format', 'text/x-wiki', 'content_model', 'wikitext', 'content_sha1', 'abc123def456', 'content_size', 1024 )), true, 'enwiki'); ``` -- 3. Add new column to struct within map ```sql ALTER TABLE test_map_struct_table ADD COLUMN revision_content_slots.value.content_origin_rev_id BIGINT; ``` -- 4. Attempt to read data → FAILS ```sql SELECT page_id, revision_content_slots['main'].content_origin_rev_id FROM test_map_struct_table WHERE page_id = 12345; ``` This produces the following error (truncated): ``` java.lang.IllegalArgumentException: [revision_content_slots, key_value, key] required binary key (STRING) = 5 is not in the store: [[page_id] required int64 page_id = 1] 1 ``` Other queries (e.g., SELECT *) succeed. ⸻ Questions: 1. Is this expected behavior? Should adding columns to structs within maps work, or is this a known limitation? 2. Field ID management: Am I correct that Parquet files don’t store Iceberg field IDs, so resolution here is name-based? 3. Error clarity: The error message seems truncated — could it be improved to include the full schema context? 4. Are there recommended workarounds, such as: • Creating a table with the new schema and moving data to it? • Avoiding schema evolution in map-contained structs? • Using specific configs or formats that handle this better? I can provide a patch after discussing the feasibility with you. Thanks a lot for your guidance! ### Willingness to contribute - [ ] I can contribute a fix for this bug independently - [x] I would be willing to contribute a fix for this bug with guidance from the Iceberg community - [ ] I cannot contribute a fix for this bug at this time -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
