From 5d9ee2e5ed165a5af867bdce47b637c327051a66 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 14:27:57 -0500 Subject: [PATCH 01/13] Add PartitionSpec to FileScanTask and handling in RecordBatchTransformer. This resolves ~50 tests in the spark-extensions Iceberg Java suite. --- crates/iceberg/src/arrow/delete_filter.rs | 6 + crates/iceberg/src/arrow/reader.rs | 843 +++++------------- .../src/arrow/record_batch_transformer.rs | 803 ++++++++++++++++- crates/iceberg/src/scan/context.rs | 6 + crates/iceberg/src/scan/mod.rs | 6 + crates/iceberg/src/scan/task.rs | 19 +- 6 files changed, 1019 insertions(+), 664 deletions(-) diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index b853baa993..4a0702c130 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -339,6 +339,9 @@ pub(crate) mod tests { project_field_ids: vec![], predicate: None, deletes: vec![pos_del_1, pos_del_2.clone()], + partition: None, + partition_spec_id: None, + partition_spec: None, }, FileScanTask { start: 0, @@ -350,6 +353,9 @@ pub(crate) mod tests { project_field_ids: vec![], predicate: None, deletes: vec![pos_del_3], + partition: None, + partition_spec_id: None, + partition_spec: None, }, ]; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index fed8f19c05..4ad8331171 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -236,7 +236,18 @@ impl ArrowReader { // that come back from the file, such as type promotion, default column insertion // and column re-ordering let mut record_batch_transformer = - RecordBatchTransformer::build(task.schema_ref(), task.project_field_ids()); + if task.partition_spec.is_some() && task.partition.is_some() { + // Use partition spec and data for proper constant identification + RecordBatchTransformer::build_with_partition_data( + task.schema_ref(), + task.project_field_ids(), + task.partition_spec.clone(), + task.partition.clone(), + ) + } else { + // Fallback to build without partition data + RecordBatchTransformer::build(task.schema_ref(), task.project_field_ids()) + }; if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); @@ -437,17 +448,10 @@ impl ArrowReader { // selected row group selected_row_groups_idx += 1; } else { - // Advance iterator past all deletes in the skipped row group. - // advance_to() positions the iterator to the first delete >= next_row_group_base_idx. - // However, if our cached next_deleted_row_idx_opt is in the skipped range, - // we need to call next() to update the cache with the newly positioned value. + // remove any positional deletes from the skipped page so that + // `positional.deletes.min()` can be used delete_vector_iter.advance_to(next_row_group_base_idx); - // Only update the cache if the cached value is stale (in the skipped range) - if let Some(cached_idx) = next_deleted_row_idx_opt { - if cached_idx < next_row_group_base_idx { - next_deleted_row_idx_opt = delete_vector_iter.next(); - } - } + next_deleted_row_idx_opt = delete_vector_iter.next(); // still increment the current page base index but then skip to the next row group // in the file @@ -462,7 +466,6 @@ impl ArrowReader { // the remainder of this row group and skip to the next row group if next_deleted_row_idx >= next_row_group_base_idx { results.push(RowSelector::select(row_group_num_rows as usize)); - current_row_group_base_idx += row_group_num_rows; continue; } @@ -472,7 +475,6 @@ impl ArrowReader { // If there are no more pos deletes, add a selector for the entirety of this row group. _ => { results.push(RowSelector::select(row_group_num_rows as usize)); - current_row_group_base_idx += row_group_num_rows; continue; } }; @@ -895,7 +897,8 @@ fn build_field_id_map(parquet_schema: &SchemaDescriptor) -> Result SchemaRef { Arc::new( @@ -1942,12 +1943,15 @@ message schema { start: 0, length: 0, record_count: None, - data_file_path: format!("{table_location}/1.parquet"), + data_file_path: format!("{}/1.parquet", table_location), data_file_format: DataFileFormat::Parquet, schema: schema.clone(), project_field_ids: vec![1], predicate: Some(predicate.bind(schema, true).unwrap()), deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, })] .into_iter(), )) as FileScanTaskStream; @@ -2266,6 +2270,9 @@ message schema { project_field_ids: vec![1], predicate: None, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, }; // Task 2: read the second and third row groups @@ -2279,6 +2286,9 @@ message schema { project_field_ids: vec![1], predicate: None, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, }; let tasks1 = Box::pin(futures::stream::iter(vec![Ok(task1)])) as FileScanTaskStream; @@ -2307,16 +2317,21 @@ message schema { .unwrap(); let total_rows_task2: usize = result2.iter().map(|b| b.num_rows()).sum(); - println!("Task 2 (bytes {rg1_start}-{file_end}) returned {total_rows_task2} rows"); + println!( + "Task 2 (bytes {}-{}) returned {} rows", + rg1_start, file_end, total_rows_task2 + ); assert_eq!( total_rows_task1, 100, - "Task 1 should read only the first row group (100 rows), but got {total_rows_task1} rows" + "Task 1 should read only the first row group (100 rows), but got {} rows", + total_rows_task1 ); assert_eq!( total_rows_task2, 200, - "Task 2 should read only the second+third row groups (200 rows), but got {total_rows_task2} rows" + "Task 2 should read only the second+third row groups (200 rows), but got {} rows", + total_rows_task2 ); // Verify the actual data values are correct (not just the row count) @@ -2327,7 +2342,7 @@ message schema { .as_primitive::(); let first_val = id_col.value(0); let last_val = id_col.value(id_col.len() - 1); - println!("Task 1 data range: {first_val} to {last_val}"); + println!("Task 1 data range: {} to {}", first_val, last_val); assert_eq!(first_val, 0, "Task 1 should start with id=0"); assert_eq!(last_val, 99, "Task 1 should end with id=99"); @@ -2339,7 +2354,7 @@ message schema { .column(0) .as_primitive::(); let first_val = id_col.value(0); - println!("Task 2 first value: {first_val}"); + println!("Task 2 first value: {}", first_val); assert_eq!(first_val, 100, "Task 2 should start with id=100, not id=0"); } @@ -2397,12 +2412,15 @@ message schema { start: 0, length: 0, record_count: None, - data_file_path: format!("{table_location}/old_file.parquet"), + data_file_path: format!("{}/old_file.parquet", table_location), data_file_format: DataFileFormat::Parquet, schema: new_schema.clone(), project_field_ids: vec![1, 2], // Request both columns 'a' and 'b' predicate: None, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, })] .into_iter(), )) as FileScanTaskStream; @@ -2438,602 +2456,6 @@ message schema { assert!(col_b.is_null(2)); } - /// Test for bug where position deletes in later row groups are not applied correctly. - /// - /// When a file has multiple row groups and a position delete targets a row in a later - /// row group, the `build_deletes_row_selection` function had a bug where it would - /// fail to increment `current_row_group_base_idx` when skipping row groups. - /// - /// This test creates: - /// - A data file with 200 rows split into 2 row groups (0-99, 100-199) - /// - A position delete file that deletes row 199 (last row in second row group) - /// - /// Expected behavior: Should return 199 rows (with id=200 deleted) - /// Bug behavior: Returns 200 rows (delete is not applied) - /// - /// This bug was discovered while running Apache Spark + Apache Iceberg integration tests - /// through DataFusion Comet. The following Iceberg Java tests failed due to this bug: - /// - `org.apache.iceberg.spark.extensions.TestMergeOnReadDelete::testDeleteWithMultipleRowGroupsParquet` - /// - `org.apache.iceberg.spark.extensions.TestMergeOnReadUpdate::testUpdateWithMultipleRowGroupsParquet` - #[tokio::test] - async fn test_position_delete_across_multiple_row_groups() { - use arrow_array::{Int32Array, Int64Array}; - use parquet::file::reader::{FileReader, SerializedFileReader}; - - // Field IDs for positional delete schema - const FIELD_ID_POSITIONAL_DELETE_FILE_PATH: u64 = 2147483546; - const FIELD_ID_POSITIONAL_DELETE_POS: u64 = 2147483545; - - let tmp_dir = TempDir::new().unwrap(); - let table_location = tmp_dir.path().to_str().unwrap().to_string(); - - // Create table schema with a single 'id' column - let table_schema = Arc::new( - Schema::builder() - .with_schema_id(1) - .with_fields(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - ]) - .build() - .unwrap(), - ); - - let arrow_schema = Arc::new(ArrowSchema::new(vec![ - Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "1".to_string(), - )])), - ])); - - // Step 1: Create data file with 200 rows in 2 row groups - // Row group 0: rows 0-99 (ids 1-100) - // Row group 1: rows 100-199 (ids 101-200) - let data_file_path = format!("{}/data.parquet", &table_location); - - let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( - Int32Array::from_iter_values(1..=100), - )]) - .unwrap(); - - let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( - Int32Array::from_iter_values(101..=200), - )]) - .unwrap(); - - // Force each batch into its own row group - let props = WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .set_max_row_group_size(100) - .build(); - - let file = File::create(&data_file_path).unwrap(); - let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); - writer.write(&batch1).expect("Writing batch 1"); - writer.write(&batch2).expect("Writing batch 2"); - writer.close().unwrap(); - - // Verify we created 2 row groups - let verify_file = File::open(&data_file_path).unwrap(); - let verify_reader = SerializedFileReader::new(verify_file).unwrap(); - assert_eq!( - verify_reader.metadata().num_row_groups(), - 2, - "Should have 2 row groups" - ); - - // Step 2: Create position delete file that deletes row 199 (id=200, last row in row group 1) - let delete_file_path = format!("{}/deletes.parquet", &table_location); - - let delete_schema = Arc::new(ArrowSchema::new(vec![ - Field::new("file_path", DataType::Utf8, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - FIELD_ID_POSITIONAL_DELETE_FILE_PATH.to_string(), - )])), - Field::new("pos", DataType::Int64, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - FIELD_ID_POSITIONAL_DELETE_POS.to_string(), - )])), - ])); - - // Delete row at position 199 (0-indexed, so it's the last row: id=200) - let delete_batch = RecordBatch::try_new(delete_schema.clone(), vec![ - Arc::new(StringArray::from_iter_values(vec![data_file_path.clone()])), - Arc::new(Int64Array::from_iter_values(vec![199i64])), - ]) - .unwrap(); - - let delete_props = WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .build(); - - let delete_file = File::create(&delete_file_path).unwrap(); - let mut delete_writer = - ArrowWriter::try_new(delete_file, delete_schema, Some(delete_props)).unwrap(); - delete_writer.write(&delete_batch).unwrap(); - delete_writer.close().unwrap(); - - // Step 3: Read the data file with the delete applied - let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); - let reader = ArrowReaderBuilder::new(file_io).build(); - - let task = FileScanTask { - start: 0, - length: 0, - record_count: Some(200), - data_file_path: data_file_path.clone(), - data_file_format: DataFileFormat::Parquet, - schema: table_schema.clone(), - project_field_ids: vec![1], - predicate: None, - deletes: vec![FileScanTaskDeleteFile { - file_path: delete_file_path, - file_type: DataContentType::PositionDeletes, - partition_spec_id: 0, - equality_ids: None, - }], - }; - - let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; - let result = reader - .read(tasks) - .unwrap() - .try_collect::>() - .await - .unwrap(); - - // Step 4: Verify we got 199 rows (not 200) - let total_rows: usize = result.iter().map(|b| b.num_rows()).sum(); - - println!("Total rows read: {}", total_rows); - println!("Expected: 199 rows (deleted row 199 which had id=200)"); - - // This assertion will FAIL before the fix and PASS after the fix - assert_eq!( - total_rows, 199, - "Expected 199 rows after deleting row 199, but got {} rows. \ - The bug causes position deletes in later row groups to be ignored.", - total_rows - ); - - // Verify the deleted row (id=200) is not present - let all_ids: Vec = result - .iter() - .flat_map(|batch| { - batch - .column(0) - .as_primitive::() - .values() - .iter() - .copied() - }) - .collect(); - - assert!( - !all_ids.contains(&200), - "Row with id=200 should be deleted but was found in results" - ); - - // Verify we have all other ids (1-199) - let expected_ids: Vec = (1..=199).collect(); - assert_eq!( - all_ids, expected_ids, - "Should have ids 1-199 but got different values" - ); - } - - /// Test for bug where position deletes are lost when skipping unselected row groups. - /// - /// This is a variant of `test_position_delete_across_multiple_row_groups` that exercises - /// the row group selection code path (`selected_row_groups: Some([...])`). - /// - /// When a file has multiple row groups and only some are selected for reading, - /// the `build_deletes_row_selection` function must correctly skip over deletes in - /// unselected row groups WITHOUT consuming deletes that belong to selected row groups. - /// - /// This test creates: - /// - A data file with 200 rows split into 2 row groups (0-99, 100-199) - /// - A position delete file that deletes row 199 (last row in second row group) - /// - Row group selection that reads ONLY row group 1 (rows 100-199) - /// - /// Expected behavior: Should return 99 rows (with row 199 deleted) - /// Bug behavior: Returns 100 rows (delete is lost when skipping row group 0) - /// - /// The bug occurs when processing row group 0 (unselected): - /// ```rust - /// delete_vector_iter.advance_to(next_row_group_base_idx); // Position at first delete >= 100 - /// next_deleted_row_idx_opt = delete_vector_iter.next(); // BUG: Consumes delete at 199! - /// ``` - /// - /// The fix is to NOT call `next()` after `advance_to()` when skipping unselected row groups, - /// because `advance_to()` already positions the iterator correctly without consuming elements. - #[tokio::test] - async fn test_position_delete_with_row_group_selection() { - use arrow_array::{Int32Array, Int64Array}; - use parquet::file::reader::{FileReader, SerializedFileReader}; - - // Field IDs for positional delete schema - const FIELD_ID_POSITIONAL_DELETE_FILE_PATH: u64 = 2147483546; - const FIELD_ID_POSITIONAL_DELETE_POS: u64 = 2147483545; - - let tmp_dir = TempDir::new().unwrap(); - let table_location = tmp_dir.path().to_str().unwrap().to_string(); - - // Create table schema with a single 'id' column - let table_schema = Arc::new( - Schema::builder() - .with_schema_id(1) - .with_fields(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - ]) - .build() - .unwrap(), - ); - - let arrow_schema = Arc::new(ArrowSchema::new(vec![ - Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "1".to_string(), - )])), - ])); - - // Step 1: Create data file with 200 rows in 2 row groups - // Row group 0: rows 0-99 (ids 1-100) - // Row group 1: rows 100-199 (ids 101-200) - let data_file_path = format!("{}/data.parquet", &table_location); - - let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( - Int32Array::from_iter_values(1..=100), - )]) - .unwrap(); - - let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( - Int32Array::from_iter_values(101..=200), - )]) - .unwrap(); - - // Force each batch into its own row group - let props = WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .set_max_row_group_size(100) - .build(); - - let file = File::create(&data_file_path).unwrap(); - let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); - writer.write(&batch1).expect("Writing batch 1"); - writer.write(&batch2).expect("Writing batch 2"); - writer.close().unwrap(); - - // Verify we created 2 row groups - let verify_file = File::open(&data_file_path).unwrap(); - let verify_reader = SerializedFileReader::new(verify_file).unwrap(); - assert_eq!( - verify_reader.metadata().num_row_groups(), - 2, - "Should have 2 row groups" - ); - - // Step 2: Create position delete file that deletes row 199 (id=200, last row in row group 1) - let delete_file_path = format!("{}/deletes.parquet", &table_location); - - let delete_schema = Arc::new(ArrowSchema::new(vec![ - Field::new("file_path", DataType::Utf8, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - FIELD_ID_POSITIONAL_DELETE_FILE_PATH.to_string(), - )])), - Field::new("pos", DataType::Int64, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - FIELD_ID_POSITIONAL_DELETE_POS.to_string(), - )])), - ])); - - // Delete row at position 199 (0-indexed, so it's the last row: id=200) - let delete_batch = RecordBatch::try_new(delete_schema.clone(), vec![ - Arc::new(StringArray::from_iter_values(vec![data_file_path.clone()])), - Arc::new(Int64Array::from_iter_values(vec![199i64])), - ]) - .unwrap(); - - let delete_props = WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .build(); - - let delete_file = File::create(&delete_file_path).unwrap(); - let mut delete_writer = - ArrowWriter::try_new(delete_file, delete_schema, Some(delete_props)).unwrap(); - delete_writer.write(&delete_batch).unwrap(); - delete_writer.close().unwrap(); - - // Step 3: Get byte ranges to read ONLY row group 1 (rows 100-199) - // This exercises the row group selection code path where row group 0 is skipped - let metadata_file = File::open(&data_file_path).unwrap(); - let metadata_reader = SerializedFileReader::new(metadata_file).unwrap(); - let metadata = metadata_reader.metadata(); - - let row_group_0 = metadata.row_group(0); - let row_group_1 = metadata.row_group(1); - - let rg0_start = 4u64; // Parquet files start with 4-byte magic "PAR1" - let rg1_start = rg0_start + row_group_0.compressed_size() as u64; - let rg1_length = row_group_1.compressed_size() as u64; - - println!( - "Row group 0: starts at byte {}, {} bytes compressed", - rg0_start, - row_group_0.compressed_size() - ); - println!( - "Row group 1: starts at byte {}, {} bytes compressed", - rg1_start, - row_group_1.compressed_size() - ); - - let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); - let reader = ArrowReaderBuilder::new(file_io).build(); - - // Create FileScanTask that reads ONLY row group 1 via byte range filtering - let task = FileScanTask { - start: rg1_start, - length: rg1_length, - record_count: Some(100), // Row group 1 has 100 rows - data_file_path: data_file_path.clone(), - data_file_format: DataFileFormat::Parquet, - schema: table_schema.clone(), - project_field_ids: vec![1], - predicate: None, - deletes: vec![FileScanTaskDeleteFile { - file_path: delete_file_path, - file_type: DataContentType::PositionDeletes, - partition_spec_id: 0, - equality_ids: None, - }], - }; - - let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; - let result = reader - .read(tasks) - .unwrap() - .try_collect::>() - .await - .unwrap(); - - // Step 4: Verify we got 99 rows (not 100) - // Row group 1 has 100 rows (ids 101-200), minus 1 delete (id=200) = 99 rows - let total_rows: usize = result.iter().map(|b| b.num_rows()).sum(); - - println!("Total rows read from row group 1: {}", total_rows); - println!("Expected: 99 rows (row group 1 has 100 rows, 1 delete at position 199)"); - - // This assertion will FAIL before the fix and PASS after the fix - assert_eq!( - total_rows, 99, - "Expected 99 rows from row group 1 after deleting position 199, but got {} rows. \ - The bug causes position deletes to be lost when advance_to() is followed by next() \ - when skipping unselected row groups.", - total_rows - ); - - // Verify the deleted row (id=200) is not present - let all_ids: Vec = result - .iter() - .flat_map(|batch| { - batch - .column(0) - .as_primitive::() - .values() - .iter() - .copied() - }) - .collect(); - - assert!( - !all_ids.contains(&200), - "Row with id=200 should be deleted but was found in results" - ); - - // Verify we have ids 101-199 (not 101-200) - let expected_ids: Vec = (101..=199).collect(); - assert_eq!( - all_ids, expected_ids, - "Should have ids 101-199 but got different values" - ); - } - /// Test for bug where stale cached delete causes infinite loop when skipping row groups. - /// - /// This test exposes the inverse scenario of `test_position_delete_with_row_group_selection`: - /// - Position delete targets a row in the SKIPPED row group (not the selected one) - /// - After calling advance_to(), the cached delete index is stale - /// - Without updating the cache, the code enters an infinite loop - /// - /// This test creates: - /// - A data file with 200 rows split into 2 row groups (0-99, 100-199) - /// - A position delete file that deletes row 0 (first row in SKIPPED row group 0) - /// - Row group selection that reads ONLY row group 1 (rows 100-199) - /// - /// The bug occurs when skipping row group 0: - /// ```rust - /// let mut next_deleted_row_idx_opt = delete_vector_iter.next(); // Some(0) - /// // ... skip to row group 1 ... - /// delete_vector_iter.advance_to(100); // Iterator advances past delete at 0 - /// // BUG: next_deleted_row_idx_opt is still Some(0) - STALE! - /// // When processing row group 1: - /// // current_idx = 100, next_deleted_row_idx = 0, next_row_group_base_idx = 200 - /// // Loop condition: 0 < 200 (true) - /// // But: current_idx (100) > next_deleted_row_idx (0) - /// // And: current_idx (100) != next_deleted_row_idx (0) - /// // Neither branch executes -> INFINITE LOOP! - /// ``` - /// - /// Expected behavior: Should return 100 rows (delete at 0 doesn't affect row group 1) - /// Bug behavior: Infinite loop in build_deletes_row_selection - #[tokio::test] - async fn test_position_delete_in_skipped_row_group() { - use arrow_array::{Int32Array, Int64Array}; - use parquet::file::reader::{FileReader, SerializedFileReader}; - - // Field IDs for positional delete schema - const FIELD_ID_POSITIONAL_DELETE_FILE_PATH: u64 = 2147483546; - const FIELD_ID_POSITIONAL_DELETE_POS: u64 = 2147483545; - - let tmp_dir = TempDir::new().unwrap(); - let table_location = tmp_dir.path().to_str().unwrap().to_string(); - - // Create table schema with a single 'id' column - let table_schema = Arc::new( - Schema::builder() - .with_schema_id(1) - .with_fields(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - ]) - .build() - .unwrap(), - ); - - let arrow_schema = Arc::new(ArrowSchema::new(vec![ - Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "1".to_string(), - )])), - ])); - - // Step 1: Create data file with 200 rows in 2 row groups - // Row group 0: rows 0-99 (ids 1-100) - // Row group 1: rows 100-199 (ids 101-200) - let data_file_path = format!("{}/data.parquet", &table_location); - - let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( - Int32Array::from_iter_values(1..=100), - )]) - .unwrap(); - - let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( - Int32Array::from_iter_values(101..=200), - )]) - .unwrap(); - - // Force each batch into its own row group - let props = WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .set_max_row_group_size(100) - .build(); - - let file = File::create(&data_file_path).unwrap(); - let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); - writer.write(&batch1).expect("Writing batch 1"); - writer.write(&batch2).expect("Writing batch 2"); - writer.close().unwrap(); - - // Verify we created 2 row groups - let verify_file = File::open(&data_file_path).unwrap(); - let verify_reader = SerializedFileReader::new(verify_file).unwrap(); - assert_eq!( - verify_reader.metadata().num_row_groups(), - 2, - "Should have 2 row groups" - ); - - // Step 2: Create position delete file that deletes row 0 (id=1, first row in row group 0) - let delete_file_path = format!("{}/deletes.parquet", &table_location); - - let delete_schema = Arc::new(ArrowSchema::new(vec![ - Field::new("file_path", DataType::Utf8, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - FIELD_ID_POSITIONAL_DELETE_FILE_PATH.to_string(), - )])), - Field::new("pos", DataType::Int64, false).with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - FIELD_ID_POSITIONAL_DELETE_POS.to_string(), - )])), - ])); - - // Delete row at position 0 (0-indexed, so it's the first row: id=1) - let delete_batch = RecordBatch::try_new(delete_schema.clone(), vec![ - Arc::new(StringArray::from_iter_values(vec![data_file_path.clone()])), - Arc::new(Int64Array::from_iter_values(vec![0i64])), - ]) - .unwrap(); - - let delete_props = WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .build(); - - let delete_file = File::create(&delete_file_path).unwrap(); - let mut delete_writer = - ArrowWriter::try_new(delete_file, delete_schema, Some(delete_props)).unwrap(); - delete_writer.write(&delete_batch).unwrap(); - delete_writer.close().unwrap(); - - // Step 3: Get byte ranges to read ONLY row group 1 (rows 100-199) - // This exercises the row group selection code path where row group 0 is skipped - let metadata_file = File::open(&data_file_path).unwrap(); - let metadata_reader = SerializedFileReader::new(metadata_file).unwrap(); - let metadata = metadata_reader.metadata(); - - let row_group_0 = metadata.row_group(0); - let row_group_1 = metadata.row_group(1); - - let rg0_start = 4u64; // Parquet files start with 4-byte magic "PAR1" - let rg1_start = rg0_start + row_group_0.compressed_size() as u64; - let rg1_length = row_group_1.compressed_size() as u64; - - let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); - let reader = ArrowReaderBuilder::new(file_io).build(); - - // Create FileScanTask that reads ONLY row group 1 via byte range filtering - let task = FileScanTask { - start: rg1_start, - length: rg1_length, - record_count: Some(100), // Row group 1 has 100 rows - data_file_path: data_file_path.clone(), - data_file_format: DataFileFormat::Parquet, - schema: table_schema.clone(), - project_field_ids: vec![1], - predicate: None, - deletes: vec![FileScanTaskDeleteFile { - file_path: delete_file_path, - file_type: DataContentType::PositionDeletes, - partition_spec_id: 0, - equality_ids: None, - }], - }; - - let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; - let result = reader - .read(tasks) - .unwrap() - .try_collect::>() - .await - .unwrap(); - - // Step 4: Verify we got 100 rows (all of row group 1) - // The delete at position 0 is in row group 0, which is skipped, so it doesn't affect us - let total_rows: usize = result.iter().map(|b| b.num_rows()).sum(); - - assert_eq!( - total_rows, 100, - "Expected 100 rows from row group 1 (delete at position 0 is in skipped row group 0). \ - If this hangs or fails, it indicates the cached delete index was not updated after advance_to()." - ); - - // Verify we have all ids from row group 1 (101-200) - let all_ids: Vec = result - .iter() - .flat_map(|batch| { - batch - .column(0) - .as_primitive::() - .values() - .iter() - .copied() - }) - .collect(); - - let expected_ids: Vec = (101..=200).collect(); - assert_eq!( - all_ids, expected_ids, - "Should have ids 101-200 (all of row group 1)" - ); - } - /// Test reading Parquet files without field ID metadata (e.g., migrated tables). /// This exercises the position-based fallback path. /// @@ -3094,6 +2516,9 @@ message schema { project_field_ids: vec![1, 2], predicate: None, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3188,6 +2613,9 @@ message schema { project_field_ids: vec![1, 3], predicate: None, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3271,6 +2699,9 @@ message schema { project_field_ids: vec![1, 2, 3], predicate: None, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3368,6 +2799,9 @@ message schema { project_field_ids: vec![1, 2], predicate: None, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3494,6 +2928,9 @@ message schema { project_field_ids: vec![1, 2], predicate: None, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3587,6 +3024,9 @@ message schema { project_field_ids: vec![1, 5, 2], predicate: None, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3693,6 +3133,9 @@ message schema { project_field_ids: vec![1, 2, 3], predicate: Some(predicate.bind(schema, true).unwrap()), deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3708,4 +3151,162 @@ message schema { // Should return empty results assert!(result.is_empty() || result.iter().all(|batch| batch.num_rows() == 0)); } + + /// Test bucket partitioning reads source column from data file (not partition metadata). + /// + /// This is an integration test verifying the complete ArrowReader pipeline with bucket partitioning. + /// It corresponds to TestRuntimeFiltering tests in Iceberg Java (e.g., testRenamedSourceColumnTable). + /// + /// # Iceberg Spec Requirements + /// + /// Per the Iceberg spec "Column Projection" section: + /// > "Return the value from partition metadata if an **Identity Transform** exists for the field" + /// + /// This means: + /// - Identity transforms (e.g., `identity(dept)`) use constants from partition metadata + /// - Non-identity transforms (e.g., `bucket(4, id)`) must read source columns from data files + /// - Partition metadata for bucket transforms stores bucket numbers (0-3), NOT source values + /// + /// Java's PartitionUtil.constantsMap() implements this via: + /// ```java + /// if (field.transform().isIdentity()) { + /// idToConstant.put(field.sourceId(), converted); + /// } + /// ``` + /// + /// # What This Test Verifies + /// + /// This test ensures the full ArrowReader → RecordBatchTransformer pipeline correctly handles + /// bucket partitioning when FileScanTask provides partition_spec and partition_data: + /// + /// - Parquet file has field_id=1 named "id" with actual data [1, 5, 9, 13] + /// - FileScanTask specifies partition_spec with bucket(4, id) and partition_data with bucket=1 + /// - RecordBatchTransformer.constants_map() excludes bucket-partitioned field from constants + /// - ArrowReader correctly reads [1, 5, 9, 13] from the data file + /// - Values are NOT replaced with constant 1 from partition metadata + /// + /// # Why This Matters + /// + /// Without correct handling: + /// - Runtime filtering would break (e.g., `WHERE id = 5` would fail) + /// - Query results would be incorrect (all rows would have id=1) + /// - Bucket partitioning would be unusable for query optimization + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" + "Partition Transforms" + /// - Java test: spark/src/test/java/.../TestRuntimeFiltering.java + /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java + #[tokio::test] + async fn test_bucket_partitioning_reads_source_column_from_file() { + use arrow_array::Int32Array; + + use crate::spec::{Literal, PartitionSpec, Struct, Transform}; + + // Iceberg schema with id and name columns + let schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: bucket(4, id) + let partition_spec = Arc::new( + PartitionSpec::builder(schema.clone()) + .with_spec_id(0) + .add_partition_field("id", "id_bucket", Transform::Bucket(4)) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: bucket value is 1 + let partition_data = Struct::from_iter(vec![Some(Literal::int(1))]); + + // Create Arrow schema with field IDs for Parquet file + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + Field::new("name", DataType::Utf8, true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])), + ])); + + // Write Parquet file with data + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + + let id_data = Arc::new(Int32Array::from(vec![1, 5, 9, 13])) as ArrayRef; + let name_data = + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie", "Dave"])) as ArrayRef; + + let to_write = + RecordBatch::try_new(arrow_schema.clone(), vec![id_data, name_data]).unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + let file = File::create(format!("{}/data.parquet", &table_location)).unwrap(); + let mut writer = ArrowWriter::try_new(file, to_write.schema(), Some(props)).unwrap(); + writer.write(&to_write).expect("Writing batch"); + writer.close().unwrap(); + + // Read the Parquet file with partition spec and data + let reader = ArrowReaderBuilder::new(file_io).build(); + let tasks = Box::pin(futures::stream::iter( + vec![Ok(FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: format!("{}/data.parquet", table_location), + data_file_format: DataFileFormat::Parquet, + schema: schema.clone(), + project_field_ids: vec![1, 2], + predicate: None, + deletes: vec![], + partition: Some(partition_data), + partition_spec_id: Some(0), + partition_spec: Some(partition_spec), + })] + .into_iter(), + )) as FileScanTaskStream; + + let result = reader + .read(tasks) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + // Verify we got the correct data + assert_eq!(result.len(), 1); + let batch = &result[0]; + + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 4); + + // The id column MUST contain actual values from the Parquet file [1, 5, 9, 13], + // NOT the constant partition value 1 + let id_col = batch + .column(0) + .as_primitive::(); + assert_eq!(id_col.value(0), 1); + assert_eq!(id_col.value(1), 5); + assert_eq!(id_col.value(2), 9); + assert_eq!(id_col.value(3), 13); + + let name_col = batch.column(1).as_string::(); + assert_eq!(name_col.value(0), "Alice"); + assert_eq!(name_col.value(1), "Bob"); + assert_eq!(name_col.value(2), "Charlie"); + assert_eq!(name_col.value(3), "Dave"); + } } diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index 5fbbbb106a..f0f5c09587 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -29,9 +29,64 @@ use arrow_schema::{ use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use crate::arrow::schema_to_arrow_schema; -use crate::spec::{Literal, PrimitiveLiteral, Schema as IcebergSchema}; +use crate::spec::{ + Literal, PartitionSpec, PrimitiveLiteral, Schema as IcebergSchema, Struct, Transform, +}; use crate::{Error, ErrorKind, Result}; +/// Build a map of field ID to constant value for identity-partitioned fields. +/// +/// This implements the behavior specified in the Iceberg spec section on "Column Projection": +/// > "Return the value from partition metadata if an Identity Transform exists for the field +/// > and the partition value is present in the `partition` struct on `data_file` object +/// > in the manifest." +/// +/// This matches Java's `PartitionUtil.constantsMap()` which only adds fields where: +/// ```java +/// if (field.transform().isIdentity()) { +/// idToConstant.put(field.sourceId(), converted); +/// } +/// ``` +/// +/// # Why only identity transforms? +/// +/// Non-identity transforms (bucket, truncate, year, month, day, hour) produce DERIVED values +/// that differ from the source column values. For example: +/// - `bucket(4, id)` produces hash values 0-3, not the actual `id` values +/// - `day(timestamp)` produces day-since-epoch integers, not the timestamp values +/// +/// These source columns MUST be read from the data file because partition metadata only +/// stores the transformed values (e.g., bucket number), not the original column values. +/// +/// # Example: Bucket Partitioning +/// +/// For a table partitioned by `bucket(4, id)`: +/// - Partition metadata stores: `id_bucket = 2` (the bucket number) +/// - Data file contains: `id = 100, 200, 300` (the actual values) +/// - Reading must use data from the file, not the constant `2` from partition metadata +/// +/// # References +/// - Iceberg spec: format/spec.md "Column Projection" section +/// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java:constantsMap() +fn constants_map( + partition_spec: &PartitionSpec, + partition_data: &Struct, +) -> HashMap { + let mut constants = HashMap::new(); + + for (pos, field) in partition_spec.fields().iter().enumerate() { + // Only identity transforms should use constant values from partition metadata + if matches!(field.transform, Transform::Identity) { + // Get the partition value for this field + if let Some(Some(Literal::Primitive(value))) = partition_data.iter().nth(pos) { + constants.insert(field.source_id, value.clone()); + } + } + } + + constants +} + /// Indicates how a particular column in a processed RecordBatch should /// be sourced. #[derive(Debug)] @@ -112,6 +167,10 @@ pub(crate) struct RecordBatchTransformer { snapshot_schema: Arc, projected_iceberg_field_ids: Vec, + // Optional partition spec and data for proper constant identification + partition_spec: Option>, + partition_data: Option, + // BatchTransform gets lazily constructed based on the schema of // the first RecordBatch we receive from the file batch_transform: Option, @@ -123,12 +182,65 @@ impl RecordBatchTransformer { pub(crate) fn build( snapshot_schema: Arc, projected_iceberg_field_ids: &[i32], + ) -> Self { + Self::build_with_partition_data(snapshot_schema, projected_iceberg_field_ids, None, None) + } + + /// Build a RecordBatchTransformer with partition spec and data for proper constant identification. + /// + /// # Why this method was added + /// + /// The gap in iceberg-rust was that `FileScanTask` had no way to pass partition information + /// to `RecordBatchTransformer`. This caused two problems: + /// + /// 1. **Incorrect handling of bucket partitioning**: Without partition spec information, + /// iceberg-rust couldn't distinguish between: + /// - Identity transforms (use constants from partition metadata) + /// - Non-identity transforms like bucket (read from data file) + /// + /// This caused bucket-partitioned source columns to be incorrectly treated as constants, + /// breaking runtime filtering and returning incorrect query results. + /// + /// 2. **Add_files field ID conflicts**: When importing Hive tables via add_files, + /// partition columns with `initial_default` values could have field IDs that conflicted + /// with data column field IDs in the Parquet file. Without detecting this conflict, + /// name-based mapping wouldn't be used, causing incorrect column reads. + /// + /// # The fix + /// + /// This method accepts `partition_spec` and `partition_data`, which are used to: + /// - Build a `constants_map` that ONLY includes identity-transformed partition fields + /// (matching Java's `PartitionUtil.constantsMap()` behavior) + /// - Detect field ID conflicts between partition columns and Parquet columns + /// - Fall back to name-based mapping when conflicts exist + /// + /// # What was changed + /// + /// To enable this fix, the following fields were added to `FileScanTask`: + /// - `partition: Option` - The partition data for this file + /// - `partition_spec_id: Option` - The spec ID for the partition + /// - `partition_spec: Option>` - The actual partition spec + /// + /// These fields should be populated by any system that reads Iceberg tables and provides + /// FileScanTasks to the ArrowReader. + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" section + /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java + /// - Java test: spark/src/test/java/.../TestRuntimeFiltering.java + pub(crate) fn build_with_partition_data( + snapshot_schema: Arc, + projected_iceberg_field_ids: &[i32], + partition_spec: Option>, + partition_data: Option, ) -> Self { let projected_iceberg_field_ids = projected_iceberg_field_ids.to_vec(); Self { snapshot_schema, projected_iceberg_field_ids, + partition_spec, + partition_data, batch_transform: None, } } @@ -167,6 +279,8 @@ impl RecordBatchTransformer { record_batch.schema_ref(), self.snapshot_schema.as_ref(), &self.projected_iceberg_field_ids, + self.partition_spec.as_ref().map(|s| s.as_ref()), + self.partition_data.as_ref(), )?); self.process_record_batch(record_batch)? @@ -185,6 +299,8 @@ impl RecordBatchTransformer { source_schema: &ArrowSchemaRef, snapshot_schema: &IcebergSchema, projected_iceberg_field_ids: &[i32], + partition_spec: Option<&PartitionSpec>, + partition_data: Option<&Struct>, ) -> Result { let mapped_unprojected_arrow_schema = Arc::new(schema_to_arrow_schema(snapshot_schema)?); let field_id_to_mapped_schema_map = @@ -205,6 +321,12 @@ impl RecordBatchTransformer { let target_schema = Arc::new(ArrowSchema::new(fields?)); + let constants_map = if let (Some(spec), Some(data)) = (partition_spec, partition_data) { + constants_map(spec, data) + } else { + HashMap::new() + }; + match Self::compare_schemas(source_schema, &target_schema) { SchemaComparison::Equivalent => Ok(BatchTransform::PassThrough), SchemaComparison::NameChangesOnly => Ok(BatchTransform::ModifySchema { target_schema }), @@ -214,6 +336,8 @@ impl RecordBatchTransformer { snapshot_schema, projected_iceberg_field_ids, field_id_to_mapped_schema_map, + constants_map, + partition_spec, )?, target_schema, }), @@ -270,57 +394,199 @@ impl RecordBatchTransformer { snapshot_schema: &IcebergSchema, projected_iceberg_field_ids: &[i32], field_id_to_mapped_schema_map: HashMap, + constants_map: HashMap, + partition_spec: Option<&PartitionSpec>, ) -> Result> { let field_id_to_source_schema_map = Self::build_field_id_to_arrow_schema_map(source_schema)?; - projected_iceberg_field_ids.iter().map(|field_id|{ - let (target_field, _) = field_id_to_mapped_schema_map.get(field_id).ok_or( - Error::new(ErrorKind::Unexpected, "could not find field in schema") - )?; - let target_type = target_field.data_type(); + // Detect field ID conflicts that require name-based mapping. + // + // Conflicts occur when fields with initial_default (from add_files or schema evolution) + // have field IDs that match Parquet field IDs, but refer to different columns. + // + // Example from add_files (Hive table import): + // Parquet file field IDs: name=1, subdept=2 + // Iceberg schema field IDs: id=1, name=2, dept=3, subdept=4 + // Partition columns (id, dept) have initial_default values + // + // Without name-based fallback, we'd incorrectly: + // - Read partition column "id" (field_id=1) from Parquet field_id=1 (which is "name") + // - Read data column "name" (field_id=2) from Parquet field_id=2 (which is "subdept") + // + // The fix: When conflicts exist, use name-based mapping for data columns. + // This matches Java's behavior in add_files procedure. + // + // Note: We do NOT treat identity-partitioned fields as conflicts. For identity partitions, + // it's NORMAL for the source column to exist in Parquet - we just use the constant from + // partition metadata instead of reading the file. + // + // See: TestAddFilesProcedure.addDataPartitionedByIdAndDept() in iceberg-java + + // Build a set of source field IDs used in NON-identity partition transforms. + // These are regular data columns that happen to be used for partitioning (e.g., bucket, truncate). + // They should be read from Parquet files normally, not treated as partition columns. + let non_identity_partition_source_ids: std::collections::HashSet = + if let Some(spec) = partition_spec { + spec.fields() + .iter() + .filter(|f| !matches!(f.transform, Transform::Identity)) + .map(|f| f.source_id) + .collect() + } else { + std::collections::HashSet::new() + }; + + let has_field_id_conflict = projected_iceberg_field_ids.iter().any(|field_id| { + let field = snapshot_schema.field_by_id(*field_id); + let has_initial_default = field.and_then(|f| f.initial_default.as_ref()).is_some(); + let in_source_schema = field_id_to_source_schema_map.contains_key(field_id); + let in_constants = constants_map.contains_key(field_id); + let is_non_identity_partition_source = + non_identity_partition_source_ids.contains(field_id); + + // A field ID conflict occurs when: + // 1. Field has initial_default (from add_files or schema evolution) + // 2. Field exists in Parquet by field ID + // 3. Field is NOT an identity-partitioned column (those use constants) + // 4. Field is NOT a source for non-identity partitioning (bucket/truncate/etc - these are data columns) + has_initial_default + && in_source_schema + && !in_constants + && !is_non_identity_partition_source + }); + + // Build name-based mapping if there's a field ID conflict + let name_to_source_schema_map: HashMap = source_schema + .fields() + .iter() + .enumerate() + .map(|(idx, field)| (field.name().clone(), (field.clone(), idx))) + .collect(); - Ok(if let Some((source_field, source_index)) = field_id_to_source_schema_map.get(field_id) { - // column present in source + projected_iceberg_field_ids + .iter() + .map(|field_id| { + let (target_field, _) = + field_id_to_mapped_schema_map + .get(field_id) + .ok_or(Error::new( + ErrorKind::Unexpected, + "could not find field in schema", + ))?; + let target_type = target_field.data_type(); - if source_field.data_type().equals_datatype(target_type) { - // no promotion required - ColumnSource::PassThrough { - source_index: *source_index - } - } else { - // promotion required - ColumnSource::Promote { + let iceberg_field = snapshot_schema.field_by_id(*field_id).ok_or(Error::new( + ErrorKind::Unexpected, + "Field not found in snapshot schema", + ))?; + + // Determine how to source this column per Iceberg spec "Column Projection" rules: + // + // Per the spec, values for field ids not present in a data file are resolved as: + // 1. "Return the value from partition metadata if an Identity Transform exists" + // 2. Use schema.name-mapping.default if present (not yet implemented) + // 3. "Return the default value if it has a defined initial-default" + // 4. "Return null in all other cases" + // + // Our implementation: + // - Step 1 is handled by constants_map (only identity transforms) + // - Steps 2-4 are handled in the else branches below + // + // Reference: format/spec.md "Column Projection" section + let column_source = if let Some(constant_value) = constants_map.get(field_id) { + // Spec rule #1: Identity-partitioned column - use constant from partition metadata + ColumnSource::Add { + value: Some(constant_value.clone()), target_type: target_type.clone(), - source_index: *source_index, } - } - } else { - // column must be added - let iceberg_field = snapshot_schema.field_by_id(*field_id).ok_or( - Error::new(ErrorKind::Unexpected, "Field not found in snapshot schema") - )?; - - let default_value = if let Some(iceberg_default_value) = - &iceberg_field.initial_default - { - let Literal::Primitive(primitive_literal) = iceberg_default_value else { - return Err(Error::new( - ErrorKind::Unexpected, - format!("Default value for column must be primitive type, but encountered {iceberg_default_value:?}") - )); - }; - Some(primitive_literal.clone()) + } else if has_field_id_conflict { + // Name-based mapping when field ID conflicts exist (add_files scenario) + if let Some((source_field, source_index)) = + name_to_source_schema_map.get(iceberg_field.name.as_str()) + { + // Column exists in Parquet by name + if source_field.data_type().equals_datatype(target_type) { + ColumnSource::PassThrough { + source_index: *source_index, + } + } else { + ColumnSource::Promote { + target_type: target_type.clone(), + source_index: *source_index, + } + } + } else { + // Column NOT in Parquet by name - use initial_default or NULL + let default_value = + iceberg_field.initial_default.as_ref().and_then(|lit| { + if let Literal::Primitive(prim) = lit { + Some(prim.clone()) + } else { + None + } + }); + ColumnSource::Add { + value: default_value, + target_type: target_type.clone(), + } + } } else { - None + // No field ID conflict - use field ID-based mapping (normal case) + let is_in_parquet = field_id_to_source_schema_map.contains_key(field_id); + + if is_in_parquet { + // Column exists in Parquet by field ID - read it + if let Some((source_field, source_index)) = + field_id_to_source_schema_map.get(field_id) + { + if source_field.data_type().equals_datatype(target_type) { + // No promotion required + ColumnSource::PassThrough { + source_index: *source_index, + } + } else { + // Promotion required + ColumnSource::Promote { + target_type: target_type.clone(), + source_index: *source_index, + } + } + } else { + // This shouldn't happen since is_in_parquet was true + let default_value = + iceberg_field.initial_default.as_ref().and_then(|lit| { + if let Literal::Primitive(prim) = lit { + Some(prim.clone()) + } else { + None + } + }); + ColumnSource::Add { + value: default_value, + target_type: target_type.clone(), + } + } + } else { + // Column NOT in Parquet by field ID - schema evolution case, use initial_default or null + let default_value = + iceberg_field.initial_default.as_ref().and_then(|lit| { + if let Literal::Primitive(prim) = lit { + Some(prim.clone()) + } else { + None + } + }); + ColumnSource::Add { + value: default_value, + target_type: target_type.clone(), + } + } }; - ColumnSource::Add { - value: default_value, - target_type: target_type.clone(), - } + Ok(column_source) }) - }).collect() + .collect() } fn build_field_id_to_arrow_schema_map( @@ -341,7 +607,7 @@ impl RecordBatchTransformer { .map_err(|e| { Error::new( ErrorKind::DataInvalid, - format!("field id not parseable as an i32: {e}"), + format!("field id not parseable as an i32: {}", e), ) })?; @@ -447,7 +713,7 @@ impl RecordBatchTransformer { (dt, _) => { return Err(Error::new( ErrorKind::Unexpected, - format!("unexpected target column type {dt}"), + format!("unexpected target column type {}", dt), )); } }) @@ -696,4 +962,457 @@ mod test { value.to_string(), )])) } + + /// Test for add_files partition column handling with field ID conflicts. + /// + /// This reproduces the scenario from Iceberg Java's TestAddFilesProcedure where: + /// - Hive-style partitioned Parquet files are imported via add_files procedure + /// - Parquet files have field IDs: name (1), subdept (2) + /// - Iceberg schema assigns different field IDs: id (1), name (2), dept (3), subdept (4) + /// - Partition columns (id, dept) have initial_default values from manifests + /// + /// Without proper handling, this would incorrectly: + /// 1. Try to read partition column "id" (field_id=1) from Parquet field_id=1 ("name") + /// 2. Read data column "name" (field_id=2) from Parquet field_id=2 ("subdept") + /// + /// The fix ensures: + /// 1. Partition columns with initial_default are ALWAYS read as constants (never from Parquet) + /// 2. Data columns use name-based mapping when field ID conflicts are detected + /// + /// See: Iceberg Java TestAddFilesProcedure.addDataPartitionedByIdAndDept() + #[test] + fn add_files_partition_columns_with_field_id_conflict() { + // Iceberg schema after add_files: id (partition), name, dept (partition), subdept + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::optional(1, "id", Type::Primitive(PrimitiveType::Int)) + .with_initial_default(Literal::int(1)) + .into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "dept", Type::Primitive(PrimitiveType::String)) + .with_initial_default(Literal::string("hr")) + .into(), + NestedField::optional(4, "subdept", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build() + .unwrap(), + ); + + // Parquet file schema: name (field_id=1), subdept (field_id=2) + // Note: Partition columns (id, dept) are NOT in the Parquet file - they're in directory paths + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("name", DataType::Utf8, true, "1"), + simple_field("subdept", DataType::Utf8, true, "2"), + ])); + + let projected_field_ids = [1, 2, 3, 4]; // id, name, dept, subdept + + let mut transformer = RecordBatchTransformer::build(snapshot_schema, &projected_field_ids); + + // Create a Parquet RecordBatch with data for: name="John Doe", subdept="communications" + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(StringArray::from(vec!["John Doe"])), + Arc::new(StringArray::from(vec!["communications"])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + // Verify the transformed RecordBatch has: + // - id=1 (from initial_default, not from Parquet) + // - name="John Doe" (from Parquet, matched by name despite field ID conflict) + // - dept="hr" (from initial_default, not from Parquet) + // - subdept="communications" (from Parquet, matched by name) + assert_eq!(result.num_columns(), 4); + assert_eq!(result.num_rows(), 1); + + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_column.value(0), 1); + + let name_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "John Doe"); + + let dept_column = result + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(dept_column.value(0), "hr"); + + let subdept_column = result + .column(3) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(subdept_column.value(0), "communications"); + } + + /// Test for bucket partitioning where source columns must be read from data files. + /// + /// This test verifies correct implementation of the Iceberg spec's "Column Projection" rules: + /// > "Return the value from partition metadata if an **Identity Transform** exists for the field" + /// + /// # Why this test is critical + /// + /// The key insight is that partition metadata stores TRANSFORMED values, not source values: + /// - For `bucket(4, id)`, partition metadata has `id_bucket = 2` (the bucket number) + /// - The actual `id` column values (100, 200, 300) are ONLY in the data file + /// + /// If iceberg-rust incorrectly treated bucket-partitioned fields as constants, it would: + /// 1. Replace all `id` values with the constant `2` from partition metadata + /// 2. Break runtime filtering (e.g., `WHERE id = 100` would match no rows) + /// 3. Return incorrect query results + /// + /// # What this test verifies + /// + /// - Bucket-partitioned fields (e.g., `bucket(4, id)`) are read from the data file + /// - The source column `id` contains actual values (100, 200, 300), not constants + /// - Java's `PartitionUtil.constantsMap()` behavior is correctly replicated: + /// ```java + /// if (field.transform().isIdentity()) { // FALSE for bucket transforms + /// idToConstant.put(field.sourceId(), converted); + /// } + /// ``` + /// + /// # Real-world impact + /// + /// This reproduces the failure scenario from Iceberg Java's TestRuntimeFiltering: + /// - Tables partitioned by `bucket(N, col)` are common for load balancing + /// - Queries filter on the source column: `SELECT * FROM tbl WHERE col = value` + /// - Runtime filtering pushes predicates down to Iceberg file scans + /// - Without this fix, the filter would match against constant partition values instead of data + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" + "Partition Transforms" + /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java + /// - Java test: spark/src/test/java/.../TestRuntimeFiltering.java + #[test] + fn bucket_partitioning_reads_source_column_from_file() { + use crate::spec::{Struct, Transform}; + + // Table schema: id (data column), name (data column), id_bucket (partition column) + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: bucket(4, id) - the id field is bucketed + let partition_spec = Arc::new( + crate::spec::PartitionSpec::builder(snapshot_schema.clone()) + .with_spec_id(0) + .add_partition_field("id", "id_bucket", Transform::Bucket(4)) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: bucket value is 2 + // In Iceberg, partition data is a Struct where each field corresponds to a partition field + let partition_data = Struct::from_iter(vec![Some(Literal::int(2))]); + + // Parquet file contains both id and name columns + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("id", DataType::Int32, false, "1"), + simple_field("name", DataType::Utf8, true, "2"), + ])); + + let projected_field_ids = [1, 2]; // id, name + + let mut transformer = RecordBatchTransformer::build_with_partition_data( + snapshot_schema, + &projected_field_ids, + Some(partition_spec), + Some(partition_data), + ); + + // Create a Parquet RecordBatch with actual data + // The id column MUST be read from here, not treated as a constant + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(Int32Array::from(vec![100, 200, 300])), + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + // Verify the transformed RecordBatch correctly reads id from the file + // (NOT as a constant from partition metadata) + assert_eq!(result.num_columns(), 2); + assert_eq!(result.num_rows(), 3); + + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + // These values MUST come from the Parquet file, not be replaced by constants + assert_eq!(id_column.value(0), 100); + assert_eq!(id_column.value(1), 200); + assert_eq!(id_column.value(2), 300); + + let name_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "Alice"); + assert_eq!(name_column.value(1), "Bob"); + assert_eq!(name_column.value(2), "Charlie"); + } + + /// Test that identity-transformed partition fields ARE treated as constants. + /// + /// This is the complement to `bucket_partitioning_reads_source_column_from_file`, + /// verifying that constants_map() correctly identifies identity-transformed + /// partition fields per the Iceberg spec. + /// + /// # Spec requirement (format/spec.md "Column Projection") + /// + /// > "Return the value from partition metadata if an Identity Transform exists for the field + /// > and the partition value is present in the `partition` struct on `data_file` object + /// > in the manifest. This allows for metadata only migrations of Hive tables." + /// + /// # Why identity transforms use constants + /// + /// Unlike bucket/truncate/year/etc., identity transforms don't modify the value: + /// - `identity(dept)` stores the actual `dept` value in partition metadata + /// - Partition metadata has `dept = "engineering"` (the real value, not a hash/bucket) + /// - This value can be used directly without reading the data file + /// + /// # Performance benefit + /// + /// For Hive migrations where partition columns aren't in data files: + /// - Partition metadata provides the column values + /// - No need to read from data files (metadata-only query optimization) + /// - Common pattern: `dept=engineering/subdept=backend/file.parquet` + /// - `dept` and `subdept` are in directory structure, not in `file.parquet` + /// - Iceberg populates these from partition metadata as constants + /// + /// # What this test verifies + /// + /// - Identity-partitioned fields use constants from partition metadata + /// - The `dept` column is populated with `"engineering"` (not read from file) + /// - Java's `PartitionUtil.constantsMap()` behavior is matched: + /// ```java + /// if (field.transform().isIdentity()) { // TRUE for identity + /// idToConstant.put(field.sourceId(), converted); + /// } + /// ``` + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" + /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java + #[test] + fn identity_partition_uses_constant_from_metadata() { + use crate::spec::{Struct, Transform}; + + // Table schema: id (data column), dept (partition column), name (data column) + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "dept", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: identity(dept) - the dept field uses identity transform + let partition_spec = Arc::new( + crate::spec::PartitionSpec::builder(snapshot_schema.clone()) + .with_spec_id(0) + .add_partition_field("dept", "dept", Transform::Identity) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: dept="engineering" + let partition_data = Struct::from_iter(vec![Some(Literal::string("engineering"))]); + + // Parquet file contains only id and name (dept is in partition path) + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("id", DataType::Int32, false, "1"), + simple_field("name", DataType::Utf8, true, "3"), + ])); + + let projected_field_ids = [1, 2, 3]; // id, dept, name + + let mut transformer = RecordBatchTransformer::build_with_partition_data( + snapshot_schema, + &projected_field_ids, + Some(partition_spec), + Some(partition_data), + ); + + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(Int32Array::from(vec![100, 200])), + Arc::new(StringArray::from(vec!["Alice", "Bob"])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + // Verify the dept column is populated with the constant from partition metadata + assert_eq!(result.num_columns(), 3); + assert_eq!(result.num_rows(), 2); + + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_column.value(0), 100); + assert_eq!(id_column.value(1), 200); + + let dept_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + // This value MUST come from partition metadata (constant) + assert_eq!(dept_column.value(0), "engineering"); + assert_eq!(dept_column.value(1), "engineering"); + + let name_column = result + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "Alice"); + assert_eq!(name_column.value(1), "Bob"); + } + + /// Test bucket partitioning with renamed source column. + /// + /// This verifies correct behavior for TestRuntimeFiltering.testRenamedSourceColumnTable() in Iceberg Java. + /// When a source column is renamed after partitioning is established, field-ID-based mapping + /// must still correctly identify the column in Parquet files. + /// + /// # Scenario + /// + /// 1. Table created with `bucket(4, id)` partitioning + /// 2. Data written to Parquet files (field_id=1, name="id") + /// 3. Column renamed: `ALTER TABLE ... RENAME COLUMN id TO row_id` + /// 4. Iceberg schema now has: field_id=1, name="row_id" + /// 5. Parquet files still have: field_id=1, name="id" + /// + /// # Expected Behavior Per Iceberg Spec + /// + /// Per the Iceberg spec "Column Projection" section and Java's PartitionUtil.constantsMap(): + /// - Bucket transforms are NON-identity, so partition metadata stores bucket numbers (0-3), not source values + /// - Source columns for non-identity transforms MUST be read from data files + /// - Field-ID-based mapping should find the column by field_id=1 (ignoring name mismatch) + /// - Runtime filtering on `row_id` should work correctly + /// + /// # What This Tests + /// + /// This test ensures that when FileScanTask provides partition_spec and partition_data: + /// - constants_map() correctly identifies that bucket(4, row_id) is NOT an identity transform + /// - The source column (field_id=1) is NOT added to constants_map + /// - Field-ID-based mapping reads actual values from the Parquet file + /// - Values [100, 200, 300] are read, not replaced with bucket constant 2 + /// + /// # References + /// - Java test: spark/src/test/java/.../TestRuntimeFiltering.java::testRenamedSourceColumnTable + /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java::constantsMap() + /// - Iceberg spec: format/spec.md "Column Projection" section + #[test] + fn test_bucket_partitioning_with_renamed_source_column() { + use crate::spec::{Struct, Transform}; + + // Iceberg schema after rename: row_id (was id), name + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "row_id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: bucket(4, row_id) - but source_id still points to field_id=1 + let partition_spec = Arc::new( + crate::spec::PartitionSpec::builder(snapshot_schema.clone()) + .with_spec_id(0) + .add_partition_field("row_id", "row_id_bucket", Transform::Bucket(4)) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: bucket value is 2 + let partition_data = Struct::from_iter(vec![Some(Literal::int(2))]); + + // Parquet file has OLD column name "id" but SAME field_id=1 + // Field-ID-based mapping should find this despite name mismatch + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("id", DataType::Int32, false, "1"), + simple_field("name", DataType::Utf8, true, "2"), + ])); + + let projected_field_ids = [1, 2]; // row_id (field_id=1), name (field_id=2) + + let mut transformer = RecordBatchTransformer::build_with_partition_data( + snapshot_schema, + &projected_field_ids, + Some(partition_spec), + Some(partition_data), + ); + + // Create a Parquet RecordBatch with actual data + // Despite column rename, data should be read via field_id=1 + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(Int32Array::from(vec![100, 200, 300])), + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + // Verify the transformed RecordBatch correctly reads data despite name mismatch + assert_eq!(result.num_columns(), 2); + assert_eq!(result.num_rows(), 3); + + let row_id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + // These values MUST come from the Parquet file via field_id=1, + // not be replaced by the bucket constant (2) + assert_eq!(row_id_column.value(0), 100); + assert_eq!(row_id_column.value(1), 200); + assert_eq!(row_id_column.value(2), 300); + + let name_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "Alice"); + assert_eq!(name_column.value(1), "Bob"); + assert_eq!(name_column.value(2), "Charlie"); + } } diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 3f7c29dbf4..2797da96aa 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -128,6 +128,12 @@ impl ManifestEntryContext { .map(|x| x.as_ref().snapshot_bound_predicate.clone()), deletes, + + // Include partition data and spec ID from manifest entry + partition: Some(self.manifest_entry.data_file.partition.clone()), + partition_spec_id: Some(self.partition_spec_id), + // TODO: Pass actual PartitionSpec through context chain for native flow + partition_spec: None, }) } } diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 6884e00b9b..137ffcfc7c 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -1777,6 +1777,9 @@ pub mod tests { record_count: Some(100), data_file_format: DataFileFormat::Parquet, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, }; test_fn(task); @@ -1791,6 +1794,9 @@ pub mod tests { record_count: None, data_file_format: DataFileFormat::Avro, deletes: vec![], + partition: None, + partition_spec_id: None, + partition_spec: None, }; test_fn(task); } diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index 32fe3ae309..7db029a389 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -18,9 +18,13 @@ use futures::stream::BoxStream; use serde::{Deserialize, Serialize}; +use std::sync::Arc; + use crate::Result; use crate::expr::BoundPredicate; -use crate::spec::{DataContentType, DataFileFormat, ManifestEntryRef, Schema, SchemaRef}; +use crate::spec::{ + DataContentType, DataFileFormat, ManifestEntryRef, PartitionSpec, Schema, SchemaRef, Struct, +}; /// A stream of [`FileScanTask`]. pub type FileScanTaskStream = BoxStream<'static, Result>; @@ -54,6 +58,19 @@ pub struct FileScanTask { /// The list of delete files that may need to be applied to this data file pub deletes: Vec, + + /// Partition data tuple from the manifest entry. + /// Schema based on the partition spec, using partition field ids for struct field ids. + #[serde(skip)] + pub partition: Option, + + /// The partition spec ID for this file. + #[serde(skip_serializing_if = "Option::is_none")] + pub partition_spec_id: Option, + + /// The partition spec for this file (for computing constants from partition data). + #[serde(skip)] + pub partition_spec: Option>, } impl FileScanTask { From ff2347a220008456ba6f20ee356af53aa9bcc287 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 14:29:05 -0500 Subject: [PATCH 02/13] format --- crates/iceberg/src/scan/task.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index 7db029a389..8b48c33c04 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use futures::stream::BoxStream; use serde::{Deserialize, Serialize}; -use std::sync::Arc; - use crate::Result; use crate::expr::BoundPredicate; use crate::spec::{ From b9e6f1ea2a25e0f26bf0cf86f82706d32901dfca Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 15:30:57 -0500 Subject: [PATCH 03/13] Put back changes that accidentally got lost from not having an updated main --- crates/iceberg/src/arrow/reader.rs | 627 ++++++++++++++++++++++++++++- 1 file changed, 620 insertions(+), 7 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 4ad8331171..5ba0e0d658 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -448,10 +448,17 @@ impl ArrowReader { // selected row group selected_row_groups_idx += 1; } else { - // remove any positional deletes from the skipped page so that - // `positional.deletes.min()` can be used + // Advance iterator past all deletes in the skipped row group. + // advance_to() positions the iterator to the first delete >= next_row_group_base_idx. + // However, if our cached next_deleted_row_idx_opt is in the skipped range, + // we need to call next() to update the cache with the newly positioned value. delete_vector_iter.advance_to(next_row_group_base_idx); - next_deleted_row_idx_opt = delete_vector_iter.next(); + // Only update the cache if the cached value is stale (in the skipped range) + if let Some(cached_idx) = next_deleted_row_idx_opt { + if cached_idx < next_row_group_base_idx { + next_deleted_row_idx_opt = delete_vector_iter.next(); + } + } // still increment the current page base index but then skip to the next row group // in the file @@ -466,6 +473,7 @@ impl ArrowReader { // the remainder of this row group and skip to the next row group if next_deleted_row_idx >= next_row_group_base_idx { results.push(RowSelector::select(row_group_num_rows as usize)); + current_row_group_base_idx += row_group_num_rows; continue; } @@ -475,6 +483,7 @@ impl ArrowReader { // If there are no more pos deletes, add a selector for the entirety of this row group. _ => { results.push(RowSelector::select(row_group_num_rows as usize)); + current_row_group_base_idx += row_group_num_rows; continue; } }; @@ -897,8 +906,7 @@ fn build_field_id_map(parquet_schema: &SchemaDescriptor) -> Result SchemaRef { Arc::new( @@ -2456,6 +2464,611 @@ message schema { assert!(col_b.is_null(2)); } + /// Test for bug where position deletes in later row groups are not applied correctly. + /// + /// When a file has multiple row groups and a position delete targets a row in a later + /// row group, the `build_deletes_row_selection` function had a bug where it would + /// fail to increment `current_row_group_base_idx` when skipping row groups. + /// + /// This test creates: + /// - A data file with 200 rows split into 2 row groups (0-99, 100-199) + /// - A position delete file that deletes row 199 (last row in second row group) + /// + /// Expected behavior: Should return 199 rows (with id=200 deleted) + /// Bug behavior: Returns 200 rows (delete is not applied) + /// + /// This bug was discovered while running Apache Spark + Apache Iceberg integration tests + /// through DataFusion Comet. The following Iceberg Java tests failed due to this bug: + /// - `org.apache.iceberg.spark.extensions.TestMergeOnReadDelete::testDeleteWithMultipleRowGroupsParquet` + /// - `org.apache.iceberg.spark.extensions.TestMergeOnReadUpdate::testUpdateWithMultipleRowGroupsParquet` + #[tokio::test] + async fn test_position_delete_across_multiple_row_groups() { + use arrow_array::{Int32Array, Int64Array}; + use parquet::file::reader::{FileReader, SerializedFileReader}; + + // Field IDs for positional delete schema + const FIELD_ID_POSITIONAL_DELETE_FILE_PATH: u64 = 2147483546; + const FIELD_ID_POSITIONAL_DELETE_POS: u64 = 2147483545; + + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + + // Create table schema with a single 'id' column + let table_schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(), + ); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + ])); + + // Step 1: Create data file with 200 rows in 2 row groups + // Row group 0: rows 0-99 (ids 1-100) + // Row group 1: rows 100-199 (ids 101-200) + let data_file_path = format!("{}/data.parquet", &table_location); + + let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(1..=100), + )]) + .unwrap(); + + let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(101..=200), + )]) + .unwrap(); + + // Force each batch into its own row group + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .set_max_row_group_size(100) + .build(); + + let file = File::create(&data_file_path).unwrap(); + let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); + writer.write(&batch1).expect("Writing batch 1"); + writer.write(&batch2).expect("Writing batch 2"); + writer.close().unwrap(); + + // Verify we created 2 row groups + let verify_file = File::open(&data_file_path).unwrap(); + let verify_reader = SerializedFileReader::new(verify_file).unwrap(); + assert_eq!( + verify_reader.metadata().num_row_groups(), + 2, + "Should have 2 row groups" + ); + + // Step 2: Create position delete file that deletes row 199 (id=200, last row in row group 1) + let delete_file_path = format!("{}/deletes.parquet", &table_location); + + let delete_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("file_path", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_FILE_PATH.to_string(), + )])), + Field::new("pos", DataType::Int64, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_POS.to_string(), + )])), + ])); + + // Delete row at position 199 (0-indexed, so it's the last row: id=200) + let delete_batch = RecordBatch::try_new(delete_schema.clone(), vec![ + Arc::new(StringArray::from_iter_values(vec![data_file_path.clone()])), + Arc::new(Int64Array::from_iter_values(vec![199i64])), + ]) + .unwrap(); + + let delete_props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let delete_file = File::create(&delete_file_path).unwrap(); + let mut delete_writer = + ArrowWriter::try_new(delete_file, delete_schema, Some(delete_props)).unwrap(); + delete_writer.write(&delete_batch).unwrap(); + delete_writer.close().unwrap(); + + // Step 3: Read the data file with the delete applied + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + let reader = ArrowReaderBuilder::new(file_io).build(); + + let task = FileScanTask { + start: 0, + length: 0, + record_count: Some(200), + data_file_path: data_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: table_schema.clone(), + project_field_ids: vec![1], + predicate: None, + deletes: vec![FileScanTaskDeleteFile { + file_path: delete_file_path, + file_type: DataContentType::PositionDeletes, + partition_spec_id: 0, + equality_ids: None, + }], + partition: None, + partition_spec_id: None, + partition_spec: None, + }; + + let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; + let result = reader + .read(tasks) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + // Step 4: Verify we got 199 rows (not 200) + let total_rows: usize = result.iter().map(|b| b.num_rows()).sum(); + + println!("Total rows read: {}", total_rows); + println!("Expected: 199 rows (deleted row 199 which had id=200)"); + + // This assertion will FAIL before the fix and PASS after the fix + assert_eq!( + total_rows, 199, + "Expected 199 rows after deleting row 199, but got {} rows. \ + The bug causes position deletes in later row groups to be ignored.", + total_rows + ); + + // Verify the deleted row (id=200) is not present + let all_ids: Vec = result + .iter() + .flat_map(|batch| { + batch + .column(0) + .as_primitive::() + .values() + .iter() + .copied() + }) + .collect(); + + assert!( + !all_ids.contains(&200), + "Row with id=200 should be deleted but was found in results" + ); + + // Verify we have all other ids (1-199) + let expected_ids: Vec = (1..=199).collect(); + assert_eq!( + all_ids, expected_ids, + "Should have ids 1-199 but got different values" + ); + } + + /// Test for bug where position deletes are lost when skipping unselected row groups. + /// + /// This is a variant of `test_position_delete_across_multiple_row_groups` that exercises + /// the row group selection code path (`selected_row_groups: Some([...])`). + /// + /// When a file has multiple row groups and only some are selected for reading, + /// the `build_deletes_row_selection` function must correctly skip over deletes in + /// unselected row groups WITHOUT consuming deletes that belong to selected row groups. + /// + /// This test creates: + /// - A data file with 200 rows split into 2 row groups (0-99, 100-199) + /// - A position delete file that deletes row 199 (last row in second row group) + /// - Row group selection that reads ONLY row group 1 (rows 100-199) + /// + /// Expected behavior: Should return 99 rows (with row 199 deleted) + /// Bug behavior: Returns 100 rows (delete is lost when skipping row group 0) + /// + /// The bug occurs when processing row group 0 (unselected): + /// ```rust + /// delete_vector_iter.advance_to(next_row_group_base_idx); // Position at first delete >= 100 + /// next_deleted_row_idx_opt = delete_vector_iter.next(); // BUG: Consumes delete at 199! + /// ``` + /// + /// The fix is to NOT call `next()` after `advance_to()` when skipping unselected row groups, + /// because `advance_to()` already positions the iterator correctly without consuming elements. + #[tokio::test] + async fn test_position_delete_with_row_group_selection() { + use arrow_array::{Int32Array, Int64Array}; + use parquet::file::reader::{FileReader, SerializedFileReader}; + + // Field IDs for positional delete schema + const FIELD_ID_POSITIONAL_DELETE_FILE_PATH: u64 = 2147483546; + const FIELD_ID_POSITIONAL_DELETE_POS: u64 = 2147483545; + + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + + // Create table schema with a single 'id' column + let table_schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(), + ); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + ])); + + // Step 1: Create data file with 200 rows in 2 row groups + // Row group 0: rows 0-99 (ids 1-100) + // Row group 1: rows 100-199 (ids 101-200) + let data_file_path = format!("{}/data.parquet", &table_location); + + let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(1..=100), + )]) + .unwrap(); + + let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(101..=200), + )]) + .unwrap(); + + // Force each batch into its own row group + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .set_max_row_group_size(100) + .build(); + + let file = File::create(&data_file_path).unwrap(); + let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); + writer.write(&batch1).expect("Writing batch 1"); + writer.write(&batch2).expect("Writing batch 2"); + writer.close().unwrap(); + + // Verify we created 2 row groups + let verify_file = File::open(&data_file_path).unwrap(); + let verify_reader = SerializedFileReader::new(verify_file).unwrap(); + assert_eq!( + verify_reader.metadata().num_row_groups(), + 2, + "Should have 2 row groups" + ); + + // Step 2: Create position delete file that deletes row 199 (id=200, last row in row group 1) + let delete_file_path = format!("{}/deletes.parquet", &table_location); + + let delete_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("file_path", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_FILE_PATH.to_string(), + )])), + Field::new("pos", DataType::Int64, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_POS.to_string(), + )])), + ])); + + // Delete row at position 199 (0-indexed, so it's the last row: id=200) + let delete_batch = RecordBatch::try_new(delete_schema.clone(), vec![ + Arc::new(StringArray::from_iter_values(vec![data_file_path.clone()])), + Arc::new(Int64Array::from_iter_values(vec![199i64])), + ]) + .unwrap(); + + let delete_props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let delete_file = File::create(&delete_file_path).unwrap(); + let mut delete_writer = + ArrowWriter::try_new(delete_file, delete_schema, Some(delete_props)).unwrap(); + delete_writer.write(&delete_batch).unwrap(); + delete_writer.close().unwrap(); + + // Step 3: Get byte ranges to read ONLY row group 1 (rows 100-199) + // This exercises the row group selection code path where row group 0 is skipped + let metadata_file = File::open(&data_file_path).unwrap(); + let metadata_reader = SerializedFileReader::new(metadata_file).unwrap(); + let metadata = metadata_reader.metadata(); + + let row_group_0 = metadata.row_group(0); + let row_group_1 = metadata.row_group(1); + + let rg0_start = 4u64; // Parquet files start with 4-byte magic "PAR1" + let rg1_start = rg0_start + row_group_0.compressed_size() as u64; + let rg1_length = row_group_1.compressed_size() as u64; + + println!( + "Row group 0: starts at byte {}, {} bytes compressed", + rg0_start, + row_group_0.compressed_size() + ); + println!( + "Row group 1: starts at byte {}, {} bytes compressed", + rg1_start, + row_group_1.compressed_size() + ); + + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + let reader = ArrowReaderBuilder::new(file_io).build(); + + // Create FileScanTask that reads ONLY row group 1 via byte range filtering + let task = FileScanTask { + start: rg1_start, + length: rg1_length, + record_count: Some(100), // Row group 1 has 100 rows + data_file_path: data_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: table_schema.clone(), + project_field_ids: vec![1], + predicate: None, + deletes: vec![FileScanTaskDeleteFile { + file_path: delete_file_path, + file_type: DataContentType::PositionDeletes, + partition_spec_id: 0, + equality_ids: None, + }], + partition: None, + partition_spec_id: None, + partition_spec: None, + }; + + let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; + let result = reader + .read(tasks) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + // Step 4: Verify we got 99 rows (not 100) + // Row group 1 has 100 rows (ids 101-200), minus 1 delete (id=200) = 99 rows + let total_rows: usize = result.iter().map(|b| b.num_rows()).sum(); + + println!("Total rows read from row group 1: {}", total_rows); + println!("Expected: 99 rows (row group 1 has 100 rows, 1 delete at position 199)"); + + // This assertion will FAIL before the fix and PASS after the fix + assert_eq!( + total_rows, 99, + "Expected 99 rows from row group 1 after deleting position 199, but got {} rows. \ + The bug causes position deletes to be lost when advance_to() is followed by next() \ + when skipping unselected row groups.", + total_rows + ); + + // Verify the deleted row (id=200) is not present + let all_ids: Vec = result + .iter() + .flat_map(|batch| { + batch + .column(0) + .as_primitive::() + .values() + .iter() + .copied() + }) + .collect(); + + assert!( + !all_ids.contains(&200), + "Row with id=200 should be deleted but was found in results" + ); + + // Verify we have ids 101-199 (not 101-200) + let expected_ids: Vec = (101..=199).collect(); + assert_eq!( + all_ids, expected_ids, + "Should have ids 101-199 but got different values" + ); + } + /// Test for bug where stale cached delete causes infinite loop when skipping row groups. + /// + /// This test exposes the inverse scenario of `test_position_delete_with_row_group_selection`: + /// - Position delete targets a row in the SKIPPED row group (not the selected one) + /// - After calling advance_to(), the cached delete index is stale + /// - Without updating the cache, the code enters an infinite loop + /// + /// This test creates: + /// - A data file with 200 rows split into 2 row groups (0-99, 100-199) + /// - A position delete file that deletes row 0 (first row in SKIPPED row group 0) + /// - Row group selection that reads ONLY row group 1 (rows 100-199) + /// + /// The bug occurs when skipping row group 0: + /// ```rust + /// let mut next_deleted_row_idx_opt = delete_vector_iter.next(); // Some(0) + /// // ... skip to row group 1 ... + /// delete_vector_iter.advance_to(100); // Iterator advances past delete at 0 + /// // BUG: next_deleted_row_idx_opt is still Some(0) - STALE! + /// // When processing row group 1: + /// // current_idx = 100, next_deleted_row_idx = 0, next_row_group_base_idx = 200 + /// // Loop condition: 0 < 200 (true) + /// // But: current_idx (100) > next_deleted_row_idx (0) + /// // And: current_idx (100) != next_deleted_row_idx (0) + /// // Neither branch executes -> INFINITE LOOP! + /// ``` + /// + /// Expected behavior: Should return 100 rows (delete at 0 doesn't affect row group 1) + /// Bug behavior: Infinite loop in build_deletes_row_selection + #[tokio::test] + async fn test_position_delete_in_skipped_row_group() { + use arrow_array::{Int32Array, Int64Array}; + use parquet::file::reader::{FileReader, SerializedFileReader}; + + // Field IDs for positional delete schema + const FIELD_ID_POSITIONAL_DELETE_FILE_PATH: u64 = 2147483546; + const FIELD_ID_POSITIONAL_DELETE_POS: u64 = 2147483545; + + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + + // Create table schema with a single 'id' column + let table_schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(), + ); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + ])); + + // Step 1: Create data file with 200 rows in 2 row groups + // Row group 0: rows 0-99 (ids 1-100) + // Row group 1: rows 100-199 (ids 101-200) + let data_file_path = format!("{}/data.parquet", &table_location); + + let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(1..=100), + )]) + .unwrap(); + + let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new( + Int32Array::from_iter_values(101..=200), + )]) + .unwrap(); + + // Force each batch into its own row group + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .set_max_row_group_size(100) + .build(); + + let file = File::create(&data_file_path).unwrap(); + let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); + writer.write(&batch1).expect("Writing batch 1"); + writer.write(&batch2).expect("Writing batch 2"); + writer.close().unwrap(); + + // Verify we created 2 row groups + let verify_file = File::open(&data_file_path).unwrap(); + let verify_reader = SerializedFileReader::new(verify_file).unwrap(); + assert_eq!( + verify_reader.metadata().num_row_groups(), + 2, + "Should have 2 row groups" + ); + + // Step 2: Create position delete file that deletes row 0 (id=1, first row in row group 0) + let delete_file_path = format!("{}/deletes.parquet", &table_location); + + let delete_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("file_path", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_FILE_PATH.to_string(), + )])), + Field::new("pos", DataType::Int64, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + FIELD_ID_POSITIONAL_DELETE_POS.to_string(), + )])), + ])); + + // Delete row at position 0 (0-indexed, so it's the first row: id=1) + let delete_batch = RecordBatch::try_new(delete_schema.clone(), vec![ + Arc::new(StringArray::from_iter_values(vec![data_file_path.clone()])), + Arc::new(Int64Array::from_iter_values(vec![0i64])), + ]) + .unwrap(); + + let delete_props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let delete_file = File::create(&delete_file_path).unwrap(); + let mut delete_writer = + ArrowWriter::try_new(delete_file, delete_schema, Some(delete_props)).unwrap(); + delete_writer.write(&delete_batch).unwrap(); + delete_writer.close().unwrap(); + + // Step 3: Get byte ranges to read ONLY row group 1 (rows 100-199) + // This exercises the row group selection code path where row group 0 is skipped + let metadata_file = File::open(&data_file_path).unwrap(); + let metadata_reader = SerializedFileReader::new(metadata_file).unwrap(); + let metadata = metadata_reader.metadata(); + + let row_group_0 = metadata.row_group(0); + let row_group_1 = metadata.row_group(1); + + let rg0_start = 4u64; // Parquet files start with 4-byte magic "PAR1" + let rg1_start = rg0_start + row_group_0.compressed_size() as u64; + let rg1_length = row_group_1.compressed_size() as u64; + + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + let reader = ArrowReaderBuilder::new(file_io).build(); + + // Create FileScanTask that reads ONLY row group 1 via byte range filtering + let task = FileScanTask { + start: rg1_start, + length: rg1_length, + record_count: Some(100), // Row group 1 has 100 rows + data_file_path: data_file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: table_schema.clone(), + project_field_ids: vec![1], + predicate: None, + deletes: vec![FileScanTaskDeleteFile { + file_path: delete_file_path, + file_type: DataContentType::PositionDeletes, + partition_spec_id: 0, + equality_ids: None, + }], + partition: None, + partition_spec_id: None, + partition_spec: None, + }; + + let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; + let result = reader + .read(tasks) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + // Step 4: Verify we got 100 rows (all of row group 1) + // The delete at position 0 is in row group 0, which is skipped, so it doesn't affect us + let total_rows: usize = result.iter().map(|b| b.num_rows()).sum(); + + assert_eq!( + total_rows, 100, + "Expected 100 rows from row group 1 (delete at position 0 is in skipped row group 0). \ + If this hangs or fails, it indicates the cached delete index was not updated after advance_to()." + ); + + // Verify we have all ids from row group 1 (101-200) + let all_ids: Vec = result + .iter() + .flat_map(|batch| { + batch + .column(0) + .as_primitive::() + .values() + .iter() + .copied() + }) + .collect(); + + let expected_ids: Vec = (101..=200).collect(); + assert_eq!( + all_ids, expected_ids, + "Should have ids 101-200 (all of row group 1)" + ); + } + /// Test reading Parquet files without field ID metadata (e.g., migrated tables). /// This exercises the position-based fallback path. /// From e7ff59769e732c5800a319e717f631af2cb546e2 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 3 Nov 2025 15:51:32 -0500 Subject: [PATCH 04/13] Format, update comments --- crates/iceberg/src/arrow/reader.rs | 4 +++- crates/iceberg/src/scan/task.rs | 15 ++++++++++----- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 5ba0e0d658..c62f73ad95 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -1656,7 +1656,9 @@ mod tests { use crate::expr::{Bind, Predicate, Reference}; use crate::io::FileIO; use crate::scan::{FileScanTask, FileScanTaskDeleteFile, FileScanTaskStream}; - use crate::spec::{DataContentType, DataFileFormat, Datum, NestedField, PrimitiveType, Schema, SchemaRef, Type}; + use crate::spec::{ + DataContentType, DataFileFormat, Datum, NestedField, PrimitiveType, Schema, SchemaRef, Type, + }; fn table_schema_simple() -> SchemaRef { Arc::new( diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index 8b48c33c04..6c587edb6f 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -59,16 +59,21 @@ pub struct FileScanTask { /// The list of delete files that may need to be applied to this data file pub deletes: Vec, - /// Partition data tuple from the manifest entry. - /// Schema based on the partition spec, using partition field ids for struct field ids. + /// Partition data from the manifest entry, used to identify which columns can use + /// constant values from partition metadata vs. reading from the data file. + /// Per the Iceberg spec, only identity-transformed partition fields should use constants. #[serde(skip)] pub partition: Option, - /// The partition spec ID for this file. - #[serde(skip_serializing_if = "Option::is_none")] + /// The partition spec ID for this file, required to look up the correct + /// partition spec and determine which fields are identity-transformed. + /// Not serialized as partition data is runtime-only and populated from manifest entries. + #[serde(skip)] pub partition_spec_id: Option, - /// The partition spec for this file (for computing constants from partition data). + /// The partition spec for this file, used to distinguish identity transforms + /// (which use partition metadata constants) from non-identity transforms like + /// bucket/truncate (which must read source columns from the data file). #[serde(skip)] pub partition_spec: Option>, } From da243bf7b7c17bc323d290d5a7a56ae127531fd4 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 4 Nov 2025 06:30:49 -0500 Subject: [PATCH 05/13] Address spec_id and serde feedback. Need to think about the name mapping. --- crates/iceberg/src/arrow/delete_filter.rs | 2 -- crates/iceberg/src/arrow/reader.rs | 15 --------- .../src/arrow/record_batch_transformer.rs | 1 - crates/iceberg/src/scan/context.rs | 3 +- crates/iceberg/src/scan/mod.rs | 2 -- crates/iceberg/src/scan/task.rs | 32 +++++++++++++------ 6 files changed, 24 insertions(+), 31 deletions(-) diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index 4a0702c130..99e5e7743d 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -340,7 +340,6 @@ pub(crate) mod tests { predicate: None, deletes: vec![pos_del_1, pos_del_2.clone()], partition: None, - partition_spec_id: None, partition_spec: None, }, FileScanTask { @@ -354,7 +353,6 @@ pub(crate) mod tests { predicate: None, deletes: vec![pos_del_3], partition: None, - partition_spec_id: None, partition_spec: None, }, ]; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index c62f73ad95..4e796bd25f 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -1960,7 +1960,6 @@ message schema { predicate: Some(predicate.bind(schema, true).unwrap()), deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, })] .into_iter(), @@ -2281,7 +2280,6 @@ message schema { predicate: None, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, }; @@ -2297,7 +2295,6 @@ message schema { predicate: None, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, }; @@ -2429,7 +2426,6 @@ message schema { predicate: None, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, })] .into_iter(), @@ -2600,7 +2596,6 @@ message schema { equality_ids: None, }], partition: None, - partition_spec_id: None, partition_spec: None, }; @@ -2818,7 +2813,6 @@ message schema { equality_ids: None, }], partition: None, - partition_spec_id: None, partition_spec: None, }; @@ -3029,7 +3023,6 @@ message schema { equality_ids: None, }], partition: None, - partition_spec_id: None, partition_spec: None, }; @@ -3132,7 +3125,6 @@ message schema { predicate: None, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, })] .into_iter(), @@ -3229,7 +3221,6 @@ message schema { predicate: None, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, })] .into_iter(), @@ -3315,7 +3306,6 @@ message schema { predicate: None, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, })] .into_iter(), @@ -3415,7 +3405,6 @@ message schema { predicate: None, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, })] .into_iter(), @@ -3544,7 +3533,6 @@ message schema { predicate: None, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, })] .into_iter(), @@ -3640,7 +3628,6 @@ message schema { predicate: None, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, })] .into_iter(), @@ -3749,7 +3736,6 @@ message schema { predicate: Some(predicate.bind(schema, true).unwrap()), deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, })] .into_iter(), @@ -3888,7 +3874,6 @@ message schema { predicate: None, deletes: vec![], partition: Some(partition_data), - partition_spec_id: Some(0), partition_spec: Some(partition_spec), })] .into_iter(), diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index f0f5c09587..0c0849badd 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -218,7 +218,6 @@ impl RecordBatchTransformer { /// /// To enable this fix, the following fields were added to `FileScanTask`: /// - `partition: Option` - The partition data for this file - /// - `partition_spec_id: Option` - The spec ID for the partition /// - `partition_spec: Option>` - The actual partition spec /// /// These fields should be populated by any system that reads Iceberg tables and provides diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 2797da96aa..9d564896d1 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -129,9 +129,8 @@ impl ManifestEntryContext { deletes, - // Include partition data and spec ID from manifest entry + // Include partition data and spec from manifest entry partition: Some(self.manifest_entry.data_file.partition.clone()), - partition_spec_id: Some(self.partition_spec_id), // TODO: Pass actual PartitionSpec through context chain for native flow partition_spec: None, }) diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 137ffcfc7c..1e9f06157c 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -1778,7 +1778,6 @@ pub mod tests { data_file_format: DataFileFormat::Parquet, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, }; test_fn(task); @@ -1795,7 +1794,6 @@ pub mod tests { data_file_format: DataFileFormat::Avro, deletes: vec![], partition: None, - partition_spec_id: None, partition_spec: None, }; test_fn(task); diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index 6c587edb6f..36bb5d391f 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use futures::stream::BoxStream; -use serde::{Deserialize, Serialize}; +use serde::{Deserialize, Serialize, Serializer}; use crate::Result; use crate::expr::BoundPredicate; @@ -29,6 +29,24 @@ use crate::spec::{ /// A stream of [`FileScanTask`]. pub type FileScanTaskStream = BoxStream<'static, Result>; +/// Serialization helper that always returns NotImplementedError. +/// Used for fields that should not be serialized but we want to be explicit about it. +fn serialize_not_implemented(_: &T, _: S) -> std::result::Result +where S: Serializer { + Err(serde::ser::Error::custom( + "Serialization not implemented for this field", + )) +} + +/// Deserialization helper that always returns NotImplementedError. +/// Used for fields that should not be deserialized but we want to be explicit about it. +fn deserialize_not_implemented<'de, D, T>(_: D) -> std::result::Result +where D: serde::Deserializer<'de> { + Err(serde::de::Error::custom( + "Deserialization not implemented for this field", + )) +} + /// A task to scan part of file. #[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct FileScanTask { @@ -62,19 +80,15 @@ pub struct FileScanTask { /// Partition data from the manifest entry, used to identify which columns can use /// constant values from partition metadata vs. reading from the data file. /// Per the Iceberg spec, only identity-transformed partition fields should use constants. - #[serde(skip)] + #[serde(serialize_with = "serialize_not_implemented")] + #[serde(deserialize_with = "deserialize_not_implemented")] pub partition: Option, - /// The partition spec ID for this file, required to look up the correct - /// partition spec and determine which fields are identity-transformed. - /// Not serialized as partition data is runtime-only and populated from manifest entries. - #[serde(skip)] - pub partition_spec_id: Option, - /// The partition spec for this file, used to distinguish identity transforms /// (which use partition metadata constants) from non-identity transforms like /// bucket/truncate (which must read source columns from the data file). - #[serde(skip)] + #[serde(serialize_with = "serialize_not_implemented")] + #[serde(deserialize_with = "deserialize_not_implemented")] pub partition_spec: Option>, } From 75cc2bcbbb23eb26abd2f93e96a84f2eee34372c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 4 Nov 2025 07:00:48 -0500 Subject: [PATCH 06/13] Add NameMapping. Still need to populate from table metadata. Want to test with DataFusion Comet first. --- crates/iceberg/src/arrow/delete_filter.rs | 2 + crates/iceberg/src/arrow/reader.rs | 16 + .../src/arrow/record_batch_transformer.rs | 471 +++++++++++------- crates/iceberg/src/scan/context.rs | 2 + crates/iceberg/src/scan/mod.rs | 2 + crates/iceberg/src/scan/task.rs | 23 +- 6 files changed, 341 insertions(+), 175 deletions(-) diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index 99e5e7743d..d9b3c168b0 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -341,6 +341,7 @@ pub(crate) mod tests { deletes: vec![pos_del_1, pos_del_2.clone()], partition: None, partition_spec: None, + name_mapping: None, }, FileScanTask { start: 0, @@ -354,6 +355,7 @@ pub(crate) mod tests { deletes: vec![pos_del_3], partition: None, partition_spec: None, + name_mapping: None, }, ]; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 4e796bd25f..6bb7faddcc 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -243,6 +243,7 @@ impl ArrowReader { task.project_field_ids(), task.partition_spec.clone(), task.partition.clone(), + task.name_mapping.clone(), ) } else { // Fallback to build without partition data @@ -1961,6 +1962,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -2281,6 +2283,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, }; // Task 2: read the second and third row groups @@ -2296,6 +2299,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, }; let tasks1 = Box::pin(futures::stream::iter(vec![Ok(task1)])) as FileScanTaskStream; @@ -2427,6 +2431,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -2597,6 +2602,7 @@ message schema { }], partition: None, partition_spec: None, + name_mapping: None, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -2814,6 +2820,7 @@ message schema { }], partition: None, partition_spec: None, + name_mapping: None, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -3024,6 +3031,7 @@ message schema { }], partition: None, partition_spec: None, + name_mapping: None, }; let tasks = Box::pin(futures::stream::iter(vec![Ok(task)])) as FileScanTaskStream; @@ -3126,6 +3134,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3222,6 +3231,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3307,6 +3317,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3406,6 +3417,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3534,6 +3546,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3629,6 +3642,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3737,6 +3751,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3875,6 +3890,7 @@ message schema { deletes: vec![], partition: Some(partition_data), partition_spec: Some(partition_spec), + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index 0c0849badd..9d6100419e 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -30,7 +30,8 @@ use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use crate::arrow::schema_to_arrow_schema; use crate::spec::{ - Literal, PartitionSpec, PrimitiveLiteral, Schema as IcebergSchema, Struct, Transform, + Literal, NameMapping, PartitionSpec, PrimitiveLiteral, Schema as IcebergSchema, Struct, + Transform, }; use crate::{Error, ErrorKind, Result}; @@ -171,6 +172,11 @@ pub(crate) struct RecordBatchTransformer { partition_spec: Option>, partition_data: Option, + // Optional name mapping for resolving field IDs from column names + // Per Iceberg spec rule #2: "Use schema.name-mapping.default metadata + // to map field id to columns without field id" + name_mapping: Option>, + // BatchTransform gets lazily constructed based on the schema of // the first RecordBatch we receive from the file batch_transform: Option, @@ -183,7 +189,7 @@ impl RecordBatchTransformer { snapshot_schema: Arc, projected_iceberg_field_ids: &[i32], ) -> Self { - Self::build_with_partition_data(snapshot_schema, projected_iceberg_field_ids, None, None) + Self::build_with_partition_data(snapshot_schema, projected_iceberg_field_ids, None, None, None) } /// Build a RecordBatchTransformer with partition spec and data for proper constant identification. @@ -208,17 +214,17 @@ impl RecordBatchTransformer { /// /// # The fix /// - /// This method accepts `partition_spec` and `partition_data`, which are used to: + /// This method accepts `partition_spec`, `partition_data`, and `name_mapping`, which are used to: /// - Build a `constants_map` that ONLY includes identity-transformed partition fields /// (matching Java's `PartitionUtil.constantsMap()` behavior) - /// - Detect field ID conflicts between partition columns and Parquet columns - /// - Fall back to name-based mapping when conflicts exist + /// - Apply name mapping when field IDs are missing or conflicting (spec rule #2) /// /// # What was changed /// /// To enable this fix, the following fields were added to `FileScanTask`: /// - `partition: Option` - The partition data for this file /// - `partition_spec: Option>` - The actual partition spec + /// - `name_mapping: Option>` - The name mapping from table metadata /// /// These fields should be populated by any system that reads Iceberg tables and provides /// FileScanTasks to the ArrowReader. @@ -232,6 +238,7 @@ impl RecordBatchTransformer { projected_iceberg_field_ids: &[i32], partition_spec: Option>, partition_data: Option, + name_mapping: Option>, ) -> Self { let projected_iceberg_field_ids = projected_iceberg_field_ids.to_vec(); @@ -240,6 +247,7 @@ impl RecordBatchTransformer { projected_iceberg_field_ids, partition_spec, partition_data, + name_mapping, batch_transform: None, } } @@ -280,6 +288,7 @@ impl RecordBatchTransformer { &self.projected_iceberg_field_ids, self.partition_spec.as_ref().map(|s| s.as_ref()), self.partition_data.as_ref(), + self.name_mapping.as_ref().map(|n| n.as_ref()), )?); self.process_record_batch(record_batch)? @@ -300,6 +309,7 @@ impl RecordBatchTransformer { projected_iceberg_field_ids: &[i32], partition_spec: Option<&PartitionSpec>, partition_data: Option<&Struct>, + name_mapping: Option<&NameMapping>, ) -> Result { let mapped_unprojected_arrow_schema = Arc::new(schema_to_arrow_schema(snapshot_schema)?); let field_id_to_mapped_schema_map = @@ -337,6 +347,7 @@ impl RecordBatchTransformer { field_id_to_mapped_schema_map, constants_map, partition_spec, + name_mapping, )?, target_schema, }), @@ -394,74 +405,16 @@ impl RecordBatchTransformer { projected_iceberg_field_ids: &[i32], field_id_to_mapped_schema_map: HashMap, constants_map: HashMap, - partition_spec: Option<&PartitionSpec>, + _partition_spec: Option<&PartitionSpec>, + name_mapping: Option<&NameMapping>, ) -> Result> { let field_id_to_source_schema_map = Self::build_field_id_to_arrow_schema_map(source_schema)?; - // Detect field ID conflicts that require name-based mapping. - // - // Conflicts occur when fields with initial_default (from add_files or schema evolution) - // have field IDs that match Parquet field IDs, but refer to different columns. - // - // Example from add_files (Hive table import): - // Parquet file field IDs: name=1, subdept=2 - // Iceberg schema field IDs: id=1, name=2, dept=3, subdept=4 - // Partition columns (id, dept) have initial_default values - // - // Without name-based fallback, we'd incorrectly: - // - Read partition column "id" (field_id=1) from Parquet field_id=1 (which is "name") - // - Read data column "name" (field_id=2) from Parquet field_id=2 (which is "subdept") - // - // The fix: When conflicts exist, use name-based mapping for data columns. - // This matches Java's behavior in add_files procedure. - // - // Note: We do NOT treat identity-partitioned fields as conflicts. For identity partitions, - // it's NORMAL for the source column to exist in Parquet - we just use the constant from - // partition metadata instead of reading the file. - // - // See: TestAddFilesProcedure.addDataPartitionedByIdAndDept() in iceberg-java - - // Build a set of source field IDs used in NON-identity partition transforms. - // These are regular data columns that happen to be used for partitioning (e.g., bucket, truncate). - // They should be read from Parquet files normally, not treated as partition columns. - let non_identity_partition_source_ids: std::collections::HashSet = - if let Some(spec) = partition_spec { - spec.fields() - .iter() - .filter(|f| !matches!(f.transform, Transform::Identity)) - .map(|f| f.source_id) - .collect() - } else { - std::collections::HashSet::new() - }; - - let has_field_id_conflict = projected_iceberg_field_ids.iter().any(|field_id| { - let field = snapshot_schema.field_by_id(*field_id); - let has_initial_default = field.and_then(|f| f.initial_default.as_ref()).is_some(); - let in_source_schema = field_id_to_source_schema_map.contains_key(field_id); - let in_constants = constants_map.contains_key(field_id); - let is_non_identity_partition_source = - non_identity_partition_source_ids.contains(field_id); - - // A field ID conflict occurs when: - // 1. Field has initial_default (from add_files or schema evolution) - // 2. Field exists in Parquet by field ID - // 3. Field is NOT an identity-partitioned column (those use constants) - // 4. Field is NOT a source for non-identity partitioning (bucket/truncate/etc - these are data columns) - has_initial_default - && in_source_schema - && !in_constants - && !is_non_identity_partition_source - }); - - // Build name-based mapping if there's a field ID conflict - let name_to_source_schema_map: HashMap = source_schema - .fields() - .iter() - .enumerate() - .map(|(idx, field)| (field.name().clone(), (field.clone(), idx))) - .collect(); + // Build name-based map for spec rule #2 (name mapping) + // This allows us to find Parquet columns by name when field IDs are missing/conflicting + let field_name_to_source_schema_map = + Self::build_field_name_to_arrow_schema_map(source_schema); projected_iceberg_field_ids .iter() @@ -480,102 +433,91 @@ impl RecordBatchTransformer { "Field not found in snapshot schema", ))?; - // Determine how to source this column per Iceberg spec "Column Projection" rules: + // Determine how to source this column per Iceberg spec "Column Projection" rules. + // + // Per the spec (https://iceberg.apache.org/spec/#column-projection): + // "Values for field ids which are not present in a data file must be resolved + // according the following rules:" // - // Per the spec, values for field ids not present in a data file are resolved as: - // 1. "Return the value from partition metadata if an Identity Transform exists" - // 2. Use schema.name-mapping.default if present (not yet implemented) + // 1. "Return the value from partition metadata if an Identity Transform exists + // for the field and the partition value is present in the partition struct + // on data_file object in the manifest." + // 2. "Use schema.name-mapping.default metadata to map field id to columns + // without field id as described below and use the column if it is present." // 3. "Return the default value if it has a defined initial-default" // 4. "Return null in all other cases" - // - // Our implementation: - // - Step 1 is handled by constants_map (only identity transforms) - // - Steps 2-4 are handled in the else branches below - // - // Reference: format/spec.md "Column Projection" section let column_source = if let Some(constant_value) = constants_map.get(field_id) { // Spec rule #1: Identity-partitioned column - use constant from partition metadata ColumnSource::Add { value: Some(constant_value.clone()), target_type: target_type.clone(), } - } else if has_field_id_conflict { - // Name-based mapping when field ID conflicts exist (add_files scenario) - if let Some((source_field, source_index)) = - name_to_source_schema_map.get(iceberg_field.name.as_str()) - { - // Column exists in Parquet by name - if source_field.data_type().equals_datatype(target_type) { - ColumnSource::PassThrough { - source_index: *source_index, - } - } else { - ColumnSource::Promote { - target_type: target_type.clone(), - source_index: *source_index, - } + } else if let Some((source_field, source_index)) = + field_id_to_source_schema_map.get(field_id) + { + // Field exists in Parquet by field ID - read it + if source_field.data_type().equals_datatype(target_type) { + ColumnSource::PassThrough { + source_index: *source_index, } } else { - // Column NOT in Parquet by name - use initial_default or NULL - let default_value = - iceberg_field.initial_default.as_ref().and_then(|lit| { - if let Literal::Primitive(prim) = lit { - Some(prim.clone()) - } else { - None - } - }); - ColumnSource::Add { - value: default_value, + ColumnSource::Promote { target_type: target_type.clone(), + source_index: *source_index, } } } else { - // No field ID conflict - use field ID-based mapping (normal case) - let is_in_parquet = field_id_to_source_schema_map.contains_key(field_id); - - if is_in_parquet { - // Column exists in Parquet by field ID - read it - if let Some((source_field, source_index)) = - field_id_to_source_schema_map.get(field_id) - { - if source_field.data_type().equals_datatype(target_type) { - // No promotion required - ColumnSource::PassThrough { - source_index: *source_index, - } - } else { - // Promotion required - ColumnSource::Promote { - target_type: target_type.clone(), - source_index: *source_index, + // Field NOT in Parquet by field ID - try spec rule #2 (name mapping) + // + // This handles scenarios like Hive table migrations where Parquet files may: + // - Have no field IDs at all + // - Have conflicting field IDs (e.g., add_files with partition columns) + // + // Per Java's implementation (ReadConf.java lines 83-85): + // ```java + // } else if (nameMapping != null) { + // typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); + // this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); + // ``` + // + // The name mapping provides a fallback: "this field ID corresponds to these possible names" + let name_mapped_column = name_mapping + .and_then(|mapping| { + // Find the mapped field for this field ID + // The NameMapping structure allows looking up by field ID to get the names + mapping.fields().iter().find_map(|mapped_field| { + if mapped_field.field_id() == Some(*field_id) { + // Try each possible name for this field + mapped_field.names().iter().find_map(|name| { + field_name_to_source_schema_map.get(name).map(|(field, idx)| { + (field.clone(), *idx) + }) + }) + } else { + None } - } + }) + }); + + if let Some((source_field, source_index)) = name_mapped_column { + // Spec rule #2: Found column via name mapping - read it from file + if source_field.data_type().equals_datatype(target_type) { + ColumnSource::PassThrough { source_index } } else { - // This shouldn't happen since is_in_parquet was true - let default_value = - iceberg_field.initial_default.as_ref().and_then(|lit| { - if let Literal::Primitive(prim) = lit { - Some(prim.clone()) - } else { - None - } - }); - ColumnSource::Add { - value: default_value, + ColumnSource::Promote { target_type: target_type.clone(), + source_index, } } } else { - // Column NOT in Parquet by field ID - schema evolution case, use initial_default or null - let default_value = - iceberg_field.initial_default.as_ref().and_then(|lit| { - if let Literal::Primitive(prim) = lit { - Some(prim.clone()) - } else { - None - } - }); + // Spec rules #3 and #4: Use initial_default if present, otherwise null + let default_value = iceberg_field.initial_default.as_ref().and_then(|lit| { + if let Literal::Primitive(prim) = lit { + Some(prim.clone()) + } else { + None + } + }); ColumnSource::Add { value: default_value, target_type: target_type.clone(), @@ -593,30 +535,50 @@ impl RecordBatchTransformer { ) -> Result> { let mut field_id_to_source_schema = HashMap::new(); for (source_field_idx, source_field) in source_schema.fields.iter().enumerate() { - let this_field_id = source_field - .metadata() - .get(PARQUET_FIELD_ID_META_KEY) - .ok_or_else(|| { - Error::new( - ErrorKind::DataInvalid, - "field ID not present in parquet metadata", - ) - })? - .parse() - .map_err(|e| { + // Check if field has a field ID in metadata + if let Some(field_id_str) = source_field.metadata().get(PARQUET_FIELD_ID_META_KEY) { + let this_field_id = field_id_str.parse().map_err(|e| { Error::new( ErrorKind::DataInvalid, format!("field id not parseable as an i32: {}", e), ) })?; - field_id_to_source_schema - .insert(this_field_id, (source_field.clone(), source_field_idx)); + field_id_to_source_schema + .insert(this_field_id, (source_field.clone(), source_field_idx)); + } + // If field doesn't have a field ID, skip it - name mapping will handle it } Ok(field_id_to_source_schema) } + /// Build a map from field name to (FieldRef, index) for name-based column resolution. + /// + /// This is used for Iceberg spec rule #2: "Use schema.name-mapping.default metadata + /// to map field id to columns without field id as described below and use the column + /// if it is present." + /// + /// Unlike `build_field_id_to_arrow_schema_map`, this method handles Parquet files + /// that may not have field IDs in their metadata. It builds a simple name-based index + /// to enable column resolution by name when field IDs are missing or conflicting. + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" section, rule #2 + /// - Java impl: ParquetSchemaUtil.applyNameMapping() + ReadConf constructor + fn build_field_name_to_arrow_schema_map( + source_schema: &SchemaRef, + ) -> HashMap { + let mut field_name_to_source_schema = HashMap::new(); + for (source_field_idx, source_field) in source_schema.fields.iter().enumerate() { + field_name_to_source_schema.insert( + source_field.name().to_string(), + (source_field.clone(), source_field_idx), + ); + } + field_name_to_source_schema + } + fn transform_columns( &self, columns: &[Arc], @@ -732,7 +694,7 @@ mod test { use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use crate::arrow::record_batch_transformer::RecordBatchTransformer; - use crate::spec::{Literal, NestedField, PrimitiveType, Schema, Type}; + use crate::spec::{Literal, NestedField, PrimitiveType, Schema, Struct, Type}; #[test] fn build_field_id_to_source_schema_map_works() { @@ -964,21 +926,21 @@ mod test { /// Test for add_files partition column handling with field ID conflicts. /// - /// This reproduces the scenario from Iceberg Java's TestAddFilesProcedure where: + /// This reproduces the scenario from Iceberg spec where: /// - Hive-style partitioned Parquet files are imported via add_files procedure - /// - Parquet files have field IDs: name (1), subdept (2) - /// - Iceberg schema assigns different field IDs: id (1), name (2), dept (3), subdept (4) - /// - Partition columns (id, dept) have initial_default values from manifests - /// - /// Without proper handling, this would incorrectly: - /// 1. Try to read partition column "id" (field_id=1) from Parquet field_id=1 ("name") - /// 2. Read data column "name" (field_id=2) from Parquet field_id=2 ("subdept") + /// - Parquet files DO NOT have field IDs (typical for Hive tables) + /// - Iceberg schema assigns field IDs: id (1), name (2), dept (3), subdept (4) + /// - Partition columns (id, dept) have initial_default values /// - /// The fix ensures: - /// 1. Partition columns with initial_default are ALWAYS read as constants (never from Parquet) - /// 2. Data columns use name-based mapping when field ID conflicts are detected + /// Per the Iceberg spec (https://iceberg.apache.org/spec/#column-projection), + /// this scenario requires `schema.name-mapping.default` from table metadata + /// to correctly map Parquet columns by name to Iceberg field IDs. /// - /// See: Iceberg Java TestAddFilesProcedure.addDataPartitionedByIdAndDept() + /// Expected behavior with name mapping: + /// 1. id=1 (from initial_default) - spec rule #3 + /// 2. name="John Doe" (from Parquet via name mapping) - spec rule #2 + /// 3. dept="hr" (from initial_default) - spec rule #3 + /// 4. subdept="communications" (from Parquet via name mapping) - spec rule #2 #[test] fn add_files_partition_columns_with_field_id_conflict() { // Iceberg schema after add_files: id (partition), name, dept (partition), subdept @@ -1000,16 +962,35 @@ mod test { .unwrap(), ); - // Parquet file schema: name (field_id=1), subdept (field_id=2) + // Parquet file schema: name, subdept (NO field IDs - typical for Hive tables) // Note: Partition columns (id, dept) are NOT in the Parquet file - they're in directory paths let parquet_schema = Arc::new(ArrowSchema::new(vec![ - simple_field("name", DataType::Utf8, true, "1"), - simple_field("subdept", DataType::Utf8, true, "2"), + Field::new("name", DataType::Utf8, true), + Field::new("subdept", DataType::Utf8, true), + ])); + + // Create name mapping to resolve field ID conflicts + // Per Iceberg spec: "Use schema.name-mapping.default metadata to map field id + // to columns without field id" + // + // The name mapping tells us: + // - Iceberg field ID 2 ("name") can be found in Parquet column "name" (even though Parquet has field_id=1) + // - Iceberg field ID 4 ("subdept") can be found in Parquet column "subdept" (even though Parquet has field_id=2) + use crate::spec::{MappedField, NameMapping}; + let name_mapping = Arc::new(NameMapping::new(vec![ + MappedField::new(Some(2), vec!["name".to_string()], vec![]), + MappedField::new(Some(4), vec!["subdept".to_string()], vec![]), ])); let projected_field_ids = [1, 2, 3, 4]; // id, name, dept, subdept - let mut transformer = RecordBatchTransformer::build(snapshot_schema, &projected_field_ids); + let mut transformer = RecordBatchTransformer::build_with_partition_data( + snapshot_schema, + &projected_field_ids, + None, + None, + Some(name_mapping), + ); // Create a Parquet RecordBatch with data for: name="John Doe", subdept="communications" let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ @@ -1139,6 +1120,7 @@ mod test { &projected_field_ids, Some(partition_spec), Some(partition_data), + None, ); // Create a Parquet RecordBatch with actual data @@ -1261,6 +1243,7 @@ mod test { &projected_field_ids, Some(partition_spec), Some(partition_data), + None, ); let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ @@ -1378,6 +1361,7 @@ mod test { &projected_field_ids, Some(partition_spec), Some(partition_data), + None, ); // Create a Parquet RecordBatch with actual data @@ -1414,4 +1398,143 @@ mod test { assert_eq!(name_column.value(1), "Bob"); assert_eq!(name_column.value(2), "Charlie"); } + + /// Comprehensive integration test that verifies all 4 Iceberg spec rules work correctly. + /// + /// Per the Iceberg spec (https://iceberg.apache.org/spec/#column-projection), + /// "Values for field ids which are not present in a data file must be resolved + /// according the following rules:" + /// + /// This test creates a scenario where each rule is exercised: + /// - Rule #1: dept (identity-partitioned) → constant from partition metadata + /// - Rule #2: data (via name mapping) → read from Parquet file by name + /// - Rule #3: category (initial_default) → use default value + /// - Rule #4: notes (no default) → return null + /// + /// # References + /// - Iceberg spec: format/spec.md "Column Projection" section + #[test] + fn test_all_four_spec_rules() { + use crate::spec::{MappedField, NameMapping, Transform}; + + // Iceberg schema with columns designed to exercise each spec rule + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + // Field in Parquet by field ID (normal case) + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + // Rule #1: Identity-partitioned field - should use partition metadata + NestedField::required(2, "dept", Type::Primitive(PrimitiveType::String)).into(), + // Rule #2: Field resolved by name mapping (no field ID in Parquet) + NestedField::required(3, "data", Type::Primitive(PrimitiveType::String)).into(), + // Rule #3: Field with initial_default + NestedField::optional(4, "category", Type::Primitive(PrimitiveType::String)) + .with_initial_default(Literal::string("default_category")) + .into(), + // Rule #4: Field with no default - should be null + NestedField::optional(5, "notes", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(), + ); + + // Partition spec: identity transform on dept + let partition_spec = Arc::new( + crate::spec::PartitionSpec::builder(snapshot_schema.clone()) + .with_spec_id(0) + .add_partition_field("dept", "dept", Transform::Identity) + .unwrap() + .build() + .unwrap(), + ); + + // Partition data: dept="engineering" + let partition_data = Struct::from_iter(vec![Some(Literal::string("engineering"))]); + + // Parquet schema: has id (with field_id=1) and data (without field ID) + // Missing: dept (in partition), category (has default), notes (no default) + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("id", DataType::Int32, false, "1"), + Field::new("data", DataType::Utf8, false), // No field ID - needs name mapping + ])); + + // Name mapping: maps field ID 3 to "data" column + let name_mapping = Arc::new(NameMapping::new(vec![MappedField::new( + Some(3), + vec!["data".to_string()], + vec![], + )])); + + let projected_field_ids = [1, 2, 3, 4, 5]; // id, dept, data, category, notes + + let mut transformer = RecordBatchTransformer::build_with_partition_data( + snapshot_schema, + &projected_field_ids, + Some(partition_spec), + Some(partition_data), + Some(name_mapping), + ); + + let parquet_batch = RecordBatch::try_new( + parquet_schema, + vec![ + Arc::new(Int32Array::from(vec![100, 200])), + Arc::new(StringArray::from(vec!["value1", "value2"])), + ], + ) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + assert_eq!(result.num_columns(), 5); + assert_eq!(result.num_rows(), 2); + + // Verify each column demonstrates the correct spec rule: + + // Normal case: id from Parquet by field ID + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_column.value(0), 100); + assert_eq!(id_column.value(1), 200); + + // Rule #1: dept from partition metadata (identity transform) + let dept_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(dept_column.value(0), "engineering"); + assert_eq!(dept_column.value(1), "engineering"); + + // Rule #2: data from Parquet via name mapping + let data_column = result + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(data_column.value(0), "value1"); + assert_eq!(data_column.value(1), "value2"); + + // Rule #3: category from initial_default + let category_column = result + .column(3) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(category_column.value(0), "default_category"); + assert_eq!(category_column.value(1), "default_category"); + + // Rule #4: notes is null (no default, not in Parquet, not in partition) + let notes_column = result + .column(4) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(notes_column.is_null(0)); + assert!(notes_column.is_null(1)); + } } diff --git a/crates/iceberg/src/scan/context.rs b/crates/iceberg/src/scan/context.rs index 9d564896d1..fe3f5c8f7e 100644 --- a/crates/iceberg/src/scan/context.rs +++ b/crates/iceberg/src/scan/context.rs @@ -133,6 +133,8 @@ impl ManifestEntryContext { partition: Some(self.manifest_entry.data_file.partition.clone()), // TODO: Pass actual PartitionSpec through context chain for native flow partition_spec: None, + // TODO: Extract name_mapping from table metadata property "schema.name-mapping.default" + name_mapping: None, }) } } diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 1e9f06157c..3e319ca062 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -1779,6 +1779,7 @@ pub mod tests { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, }; test_fn(task); @@ -1795,6 +1796,7 @@ pub mod tests { deletes: vec![], partition: None, partition_spec: None, + name_mapping: None, }; test_fn(task); } diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index 36bb5d391f..04259bd34c 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -23,7 +23,8 @@ use serde::{Deserialize, Serialize, Serializer}; use crate::Result; use crate::expr::BoundPredicate; use crate::spec::{ - DataContentType, DataFileFormat, ManifestEntryRef, PartitionSpec, Schema, SchemaRef, Struct, + DataContentType, DataFileFormat, ManifestEntryRef, NameMapping, PartitionSpec, Schema, + SchemaRef, Struct, }; /// A stream of [`FileScanTask`]. @@ -80,6 +81,8 @@ pub struct FileScanTask { /// Partition data from the manifest entry, used to identify which columns can use /// constant values from partition metadata vs. reading from the data file. /// Per the Iceberg spec, only identity-transformed partition fields should use constants. + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] #[serde(serialize_with = "serialize_not_implemented")] #[serde(deserialize_with = "deserialize_not_implemented")] pub partition: Option, @@ -87,9 +90,27 @@ pub struct FileScanTask { /// The partition spec for this file, used to distinguish identity transforms /// (which use partition metadata constants) from non-identity transforms like /// bucket/truncate (which must read source columns from the data file). + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] #[serde(serialize_with = "serialize_not_implemented")] #[serde(deserialize_with = "deserialize_not_implemented")] pub partition_spec: Option>, + + /// Name mapping from table metadata (property: schema.name-mapping.default), + /// used to resolve field IDs from column names when Parquet files lack field IDs + /// or have field ID conflicts. + /// + /// Per Iceberg spec rule #2: "Use schema.name-mapping.default metadata to map + /// field id to columns without field id". + /// + /// This is essential for scenarios like: + /// - Hive table migrations via add_files where Parquet has no field IDs + /// - Field ID conflicts where partition columns conflict with data column IDs + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(serialize_with = "serialize_not_implemented")] + #[serde(deserialize_with = "deserialize_not_implemented")] + pub name_mapping: Option>, } impl FileScanTask { From 29dde0e2b8e3ea3afe919448ebdf5936d7472f2f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 4 Nov 2025 10:13:51 -0500 Subject: [PATCH 07/13] Handle field ID conflicts in add_files with name mapping --- .../src/arrow/caching_delete_file_loader.rs | 116 ++++++++++ crates/iceberg/src/arrow/delete_filter.rs | 4 +- crates/iceberg/src/arrow/reader.rs | 43 ++-- .../src/arrow/record_batch_transformer.rs | 219 +++++++++++++----- 4 files changed, 298 insertions(+), 84 deletions(-) diff --git a/crates/iceberg/src/arrow/caching_delete_file_loader.rs b/crates/iceberg/src/arrow/caching_delete_file_loader.rs index 078635c9be..f1c4f86f23 100644 --- a/crates/iceberg/src/arrow/caching_delete_file_loader.rs +++ b/crates/iceberg/src/arrow/caching_delete_file_loader.rs @@ -788,4 +788,120 @@ mod tests { assert_eq!(data_col.value(1), "d"); assert_eq!(data_col.value(2), "g"); } + + /// Test loading a FileScanTask with BOTH positional and equality deletes. + /// Verifies the fix for the inverted condition that caused "Missing predicate for equality delete file" errors. + #[tokio::test] + async fn test_load_deletes_with_mixed_types() { + use crate::scan::FileScanTask; + use crate::spec::{DataFileFormat, Schema}; + + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path(); + let file_io = FileIO::from_path(table_location.as_os_str().to_str().unwrap()) + .unwrap() + .build() + .unwrap(); + + // Create the data file schema + let data_file_schema = Arc::new( + Schema::builder() + .with_fields(vec![ + crate::spec::NestedField::optional( + 2, + "y", + crate::spec::Type::Primitive(crate::spec::PrimitiveType::Long), + ) + .into(), + crate::spec::NestedField::optional( + 3, + "z", + crate::spec::Type::Primitive(crate::spec::PrimitiveType::Long), + ) + .into(), + ]) + .build() + .unwrap(), + ); + + // Write positional delete file + let positional_delete_schema = crate::arrow::delete_filter::tests::create_pos_del_schema(); + let file_path_values = + vec![format!("{}/data-1.parquet", table_location.to_str().unwrap()); 4]; + let file_path_col = Arc::new(StringArray::from_iter_values(&file_path_values)); + let pos_col = Arc::new(Int64Array::from_iter_values(vec![0i64, 1, 2, 3])); + + let positional_deletes_to_write = + RecordBatch::try_new(positional_delete_schema.clone(), vec![ + file_path_col, + pos_col, + ]) + .unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + + let pos_del_path = format!("{}/pos-del-mixed.parquet", table_location.to_str().unwrap()); + let file = File::create(&pos_del_path).unwrap(); + let mut writer = ArrowWriter::try_new( + file, + positional_deletes_to_write.schema(), + Some(props.clone()), + ) + .unwrap(); + writer.write(&positional_deletes_to_write).unwrap(); + writer.close().unwrap(); + + // Write equality delete file + let eq_delete_path = setup_write_equality_delete_file_1(table_location.to_str().unwrap()); + + // Create FileScanTask with BOTH positional and equality deletes + let pos_del = FileScanTaskDeleteFile { + file_path: pos_del_path, + file_type: DataContentType::PositionDeletes, + partition_spec_id: 0, + equality_ids: None, + }; + + let eq_del = FileScanTaskDeleteFile { + file_path: eq_delete_path.clone(), + file_type: DataContentType::EqualityDeletes, + partition_spec_id: 0, + equality_ids: Some(vec![2, 3]), // Only use field IDs that exist in both schemas + }; + + let file_scan_task = FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: format!("{}/data-1.parquet", table_location.to_str().unwrap()), + data_file_format: DataFileFormat::Parquet, + schema: data_file_schema.clone(), + project_field_ids: vec![2, 3], + predicate: None, + deletes: vec![pos_del, eq_del], + partition: None, + partition_spec: None, + name_mapping: None, + }; + + // Load the deletes - should handle both types without error + let delete_file_loader = CachingDeleteFileLoader::new(file_io.clone(), 10); + let delete_filter = delete_file_loader + .load_deletes(&file_scan_task.deletes, file_scan_task.schema_ref()) + .await + .unwrap() + .unwrap(); + + // Verify both delete types can be processed together + let result = delete_filter + .build_equality_delete_predicate(&file_scan_task) + .await; + assert!( + result.is_ok(), + "Failed to build equality delete predicate: {:?}", + result.err() + ); + } } diff --git a/crates/iceberg/src/arrow/delete_filter.rs b/crates/iceberg/src/arrow/delete_filter.rs index d9b3c168b0..e843cbaa95 100644 --- a/crates/iceberg/src/arrow/delete_filter.rs +++ b/crates/iceberg/src/arrow/delete_filter.rs @@ -341,7 +341,7 @@ pub(crate) mod tests { deletes: vec![pos_del_1, pos_del_2.clone()], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, }, FileScanTask { start: 0, @@ -355,7 +355,7 @@ pub(crate) mod tests { deletes: vec![pos_del_3], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, }, ]; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 6bb7faddcc..02f60bdf8a 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -234,21 +234,16 @@ impl ArrowReader { // RecordBatchTransformer performs any transformations required on the RecordBatches // that come back from the file, such as type promotion, default column insertion - // and column re-ordering - let mut record_batch_transformer = - if task.partition_spec.is_some() && task.partition.is_some() { - // Use partition spec and data for proper constant identification - RecordBatchTransformer::build_with_partition_data( - task.schema_ref(), - task.project_field_ids(), - task.partition_spec.clone(), - task.partition.clone(), - task.name_mapping.clone(), - ) - } else { - // Fallback to build without partition data - RecordBatchTransformer::build(task.schema_ref(), task.project_field_ids()) - }; + // and column re-ordering. + // Always use build_with_partition_data to ensure name_mapping is passed through, + // even when partition spec/data aren't available. + let mut record_batch_transformer = RecordBatchTransformer::build_with_partition_data( + task.schema_ref(), + task.project_field_ids(), + task.partition_spec.clone(), + task.partition.clone(), + task.name_mapping.clone(), + ); if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); @@ -1962,7 +1957,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -2431,7 +2426,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3134,7 +3129,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3231,7 +3226,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3317,7 +3312,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3417,7 +3412,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3546,7 +3541,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3642,7 +3637,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; @@ -3751,7 +3746,7 @@ message schema { deletes: vec![], partition: None, partition_spec: None, - name_mapping: None, + name_mapping: None, })] .into_iter(), )) as FileScanTaskStream; diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index 9d6100419e..a290a78881 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -37,7 +37,8 @@ use crate::{Error, ErrorKind, Result}; /// Build a map of field ID to constant value for identity-partitioned fields. /// -/// This implements the behavior specified in the Iceberg spec section on "Column Projection": +/// This implements the Iceberg spec's "Column Projection" rule #1 +/// (https://iceberg.apache.org/spec/#column-projection): /// > "Return the value from partition metadata if an Identity Transform exists for the field /// > and the partition value is present in the `partition` struct on `data_file` object /// > in the manifest." @@ -59,6 +60,23 @@ use crate::{Error, ErrorKind, Result}; /// These source columns MUST be read from the data file because partition metadata only /// stores the transformed values (e.g., bucket number), not the original column values. /// +/// # Java Implementation Reference +/// +/// This matches Java's `PartitionUtil.constantsMap()` (util/PartitionUtil.java): +/// ```java +/// public static Map constantsMap(PartitionData data, PartitionSpec spec) { +/// Map idToConstant = Maps.newHashMap(); +/// for (int pos = 0; pos < spec.fields().size(); pos += 1) { +/// PartitionField field = spec.fields().get(pos); +/// if (field.transform().isIdentity()) { // <-- ONLY identity transforms +/// Object converted = convertConstant(field.sourceId(), data.get(pos, javaClass)); +/// idToConstant.put(field.sourceId(), converted); +/// } +/// } +/// return idToConstant; +/// } +/// ``` +/// /// # Example: Bucket Partitioning /// /// For a table partitioned by `bucket(4, id)`: @@ -189,11 +207,28 @@ impl RecordBatchTransformer { snapshot_schema: Arc, projected_iceberg_field_ids: &[i32], ) -> Self { - Self::build_with_partition_data(snapshot_schema, projected_iceberg_field_ids, None, None, None) + Self::build_with_partition_data( + snapshot_schema, + projected_iceberg_field_ids, + None, + None, + None, + ) } /// Build a RecordBatchTransformer with partition spec and data for proper constant identification. /// + /// # Overview + /// + /// This method implements the Iceberg spec's "Column Projection" rules + /// (https://iceberg.apache.org/spec/#column-projection) for resolving field IDs that are + /// "not present" in a data file: + /// + /// 1. Return the value from partition metadata if an Identity Transform exists + /// 2. Use schema.name-mapping.default metadata to map field id to columns without field id + /// 3. Return the default value if it has a defined initial-default + /// 4. Return null in all other cases + /// /// # Why this method was added /// /// The gap in iceberg-rust was that `FileScanTask` had no way to pass partition information @@ -209,16 +244,29 @@ impl RecordBatchTransformer { /// /// 2. **Add_files field ID conflicts**: When importing Hive tables via add_files, /// partition columns with `initial_default` values could have field IDs that conflicted - /// with data column field IDs in the Parquet file. Without detecting this conflict, - /// name-based mapping wouldn't be used, causing incorrect column reads. + /// with data column field IDs in the Parquet file. + /// + /// Example: + /// - Parquet file written with: field_id=1→"name", field_id=2→"dept" + /// - Imported via add_files: field_id=1→"id" (partition), field_id=2→"name", field_id=3→"dept" + /// + /// When looking for field_id=1 ("id"), we find field_id=1 in the Parquet file, but it's + /// the WRONG field (it's "name"). Per the spec, the correct field (id=1, name="id") is + /// "not present" in the file and should be resolved via name mapping (rule #2) or + /// initial-default (rule #3). /// /// # The fix /// /// This method accepts `partition_spec`, `partition_data`, and `name_mapping`, which are used to: /// - Build a `constants_map` that ONLY includes identity-transformed partition fields /// (matching Java's `PartitionUtil.constantsMap()` behavior) + /// - Detect field ID conflicts by verifying both field ID AND name match (when name mapping present) /// - Apply name mapping when field IDs are missing or conflicting (spec rule #2) /// + /// This matches Java's approach (ParquetSchemaUtil.applyNameMapping, ReadConf.java lines 83-85) + /// which rewrites Parquet schema field IDs based on names before projection. Our implementation + /// detects conflicts during projection but achieves the same result. + /// /// # What was changed /// /// To enable this fix, the following fields were added to `FileScanTask`: @@ -230,8 +278,9 @@ impl RecordBatchTransformer { /// FileScanTasks to the ArrowReader. /// /// # References - /// - Iceberg spec: format/spec.md "Column Projection" section + /// - Iceberg spec: https://iceberg.apache.org/spec/#column-projection /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java + /// - Java impl: parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java /// - Java test: spark/src/test/java/.../TestRuntimeFiltering.java pub(crate) fn build_with_partition_data( snapshot_schema: Arc, @@ -446,52 +495,106 @@ impl RecordBatchTransformer { // without field id as described below and use the column if it is present." // 3. "Return the default value if it has a defined initial-default" // 4. "Return null in all other cases" + let column_source = if let Some(constant_value) = constants_map.get(field_id) { // Spec rule #1: Identity-partitioned column - use constant from partition metadata ColumnSource::Add { value: Some(constant_value.clone()), target_type: target_type.clone(), } - } else if let Some((source_field, source_index)) = - field_id_to_source_schema_map.get(field_id) - { - // Field exists in Parquet by field ID - read it - if source_field.data_type().equals_datatype(target_type) { - ColumnSource::PassThrough { - source_index: *source_index, - } - } else { - ColumnSource::Promote { - target_type: target_type.clone(), - source_index: *source_index, - } - } } else { - // Field NOT in Parquet by field ID - try spec rule #2 (name mapping) + // Check if field ID exists in Parquet, but verify it's the CORRECT field. + // + // Per the Iceberg spec (https://iceberg.apache.org/spec/#column-projection): + // "Values for field ids which are NOT PRESENT in a data file must be resolved + // according the following rules..." + // + // The key insight: In add_files scenarios (Hive table migrations), Parquet files + // may have field IDs that conflict with the Iceberg schema's field IDs: // - // This handles scenarios like Hive table migrations where Parquet files may: - // - Have no field IDs at all - // - Have conflicting field IDs (e.g., add_files with partition columns) + // Example: + // - Parquet file written with: field_id=1→"name", field_id=2→"dept" + // - Imported via add_files with: field_id=1→"id" (partition), field_id=2→"name", field_id=3→"dept" // - // Per Java's implementation (ReadConf.java lines 83-85): + // When we look for Iceberg field_id=1 ("id"), we find a field_id=1 in the Parquet file, + // but it's the WRONG field (it's "name", not "id"). The correct field (id=1, name="id") + // is NOT PRESENT in the Parquet file - it only exists as partition metadata. + // + // Per the spec: when a field is "not present", we should apply rules 2-4 (name mapping, + // initial-default, or null). + // + // Java's approach (ParquetSchemaUtil.applyNameMapping, ReadConf.java lines 83-85): + // Java REWRITES the Parquet schema's field IDs based on names BEFORE projection: // ```java - // } else if (nameMapping != null) { + // if (nameMapping != null) { // typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); + // // Now field IDs match Iceberg schema based on name mapping // this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); + // } // ``` // - // The name mapping provides a fallback: "this field ID corresponds to these possible names" - let name_mapped_column = name_mapping - .and_then(|mapping| { + // Our approach achieves the same result but detects conflicts DURING projection: + // - When name mapping is present, field ID matches alone aren't sufficient + // - We verify the field NAME also matches to ensure it's the correct field + // - If names don't match, we treat the field as "not present" and use name mapping + let field_by_id = if let Some((source_field, source_index)) = + field_id_to_source_schema_map.get(field_id) + { + let name_matches = source_field.name() == &iceberg_field.name; + + if name_mapping.is_some() && !name_matches { + // Field ID conflict detected: Parquet has this field ID but for a different field. + // The field we're looking for (this field_id + this name) is NOT PRESENT in the file. + // Per spec: treat as "not present" and fall through to name mapping (rule #2). + None + } else { + // Field ID matches and either: + // - No name mapping present (trust the field ID) + // - Names also match (correct field, use it) + if source_field.data_type().equals_datatype(target_type) { + Some(ColumnSource::PassThrough { + source_index: *source_index, + }) + } else { + Some(ColumnSource::Promote { + target_type: target_type.clone(), + source_index: *source_index, + }) + } + } + } else { + None + }; + + if let Some(source) = field_by_id { + source + } else { + // Field not found by ID, or field ID conflict detected. + // Per spec: field is "not present", apply remaining rules. + // Field NOT in Parquet by field ID - try spec rule #2 (name mapping) + // + // This handles scenarios like Hive table migrations where Parquet files may: + // - Have no field IDs at all + // - Have conflicting field IDs (e.g., add_files with partition columns) + // + // Per Java's implementation (ReadConf.java lines 83-85): + // ```java + // } else if (nameMapping != null) { + // typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); + // this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); + // ``` + // + // The name mapping provides a fallback: "this field ID corresponds to these possible names" + let name_mapped_column = name_mapping.and_then(|mapping| { // Find the mapped field for this field ID // The NameMapping structure allows looking up by field ID to get the names mapping.fields().iter().find_map(|mapped_field| { if mapped_field.field_id() == Some(*field_id) { // Try each possible name for this field mapped_field.names().iter().find_map(|name| { - field_name_to_source_schema_map.get(name).map(|(field, idx)| { - (field.clone(), *idx) - }) + field_name_to_source_schema_map + .get(name) + .map(|(field, idx)| (field.clone(), *idx)) }) } else { None @@ -499,29 +602,31 @@ impl RecordBatchTransformer { }) }); - if let Some((source_field, source_index)) = name_mapped_column { - // Spec rule #2: Found column via name mapping - read it from file - if source_field.data_type().equals_datatype(target_type) { - ColumnSource::PassThrough { source_index } + if let Some((source_field, source_index)) = name_mapped_column { + // Spec rule #2: Found column via name mapping - read it from file + if source_field.data_type().equals_datatype(target_type) { + ColumnSource::PassThrough { source_index } + } else { + ColumnSource::Promote { + target_type: target_type.clone(), + source_index, + } + } } else { - ColumnSource::Promote { + // Spec rules #3 and #4: Use initial_default if present, otherwise null + let default_value = + iceberg_field.initial_default.as_ref().and_then(|lit| { + if let Literal::Primitive(prim) = lit { + Some(prim.clone()) + } else { + None + } + }); + ColumnSource::Add { + value: default_value, target_type: target_type.clone(), - source_index, } } - } else { - // Spec rules #3 and #4: Use initial_default if present, otherwise null - let default_value = iceberg_field.initial_default.as_ref().and_then(|lit| { - if let Literal::Primitive(prim) = lit { - Some(prim.clone()) - } else { - None - } - }); - ColumnSource::Add { - value: default_value, - target_type: target_type.clone(), - } } }; @@ -1433,7 +1538,8 @@ mod test { .with_initial_default(Literal::string("default_category")) .into(), // Rule #4: Field with no default - should be null - NestedField::optional(5, "notes", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(5, "notes", Type::Primitive(PrimitiveType::String)) + .into(), ]) .build() .unwrap(), @@ -1476,13 +1582,10 @@ mod test { Some(name_mapping), ); - let parquet_batch = RecordBatch::try_new( - parquet_schema, - vec![ - Arc::new(Int32Array::from(vec![100, 200])), - Arc::new(StringArray::from(vec!["value1", "value2"])), - ], - ) + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(Int32Array::from(vec![100, 200])), + Arc::new(StringArray::from(vec!["value1", "value2"])), + ]) .unwrap(); let result = transformer.process_record_batch(parquet_batch).unwrap(); From c9538373bf10555a8baa2f2ad4c5f02d5193c424 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 4 Nov 2025 11:55:02 -0500 Subject: [PATCH 08/13] clean up comments a bit, add a new test --- .../src/arrow/record_batch_transformer.rs | 171 +++++++++++++++--- 1 file changed, 150 insertions(+), 21 deletions(-) diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index a290a78881..e586b1aa02 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -537,21 +537,20 @@ impl RecordBatchTransformer { // - When name mapping is present, field ID matches alone aren't sufficient // - We verify the field NAME also matches to ensure it's the correct field // - If names don't match, we treat the field as "not present" and use name mapping - let field_by_id = if let Some((source_field, source_index)) = - field_id_to_source_schema_map.get(field_id) - { - let name_matches = source_field.name() == &iceberg_field.name; - - if name_mapping.is_some() && !name_matches { - // Field ID conflict detected: Parquet has this field ID but for a different field. - // The field we're looking for (this field_id + this name) is NOT PRESENT in the file. - // Per spec: treat as "not present" and fall through to name mapping (rule #2). - None - } else { - // Field ID matches and either: - // - No name mapping present (trust the field ID) - // - Names also match (correct field, use it) - if source_field.data_type().equals_datatype(target_type) { + let field_by_id = field_id_to_source_schema_map + .get(field_id) + .and_then(|(source_field, source_index)| { + let name_matches = source_field.name() == &iceberg_field.name; + + if name_mapping.is_some() && !name_matches { + // Field ID conflict detected: Parquet has this field ID but for a different field. + // The field we're looking for (this field_id + this name) is NOT PRESENT in the file. + // Per spec: treat as "not present" and fall through to name mapping (rule #2). + None + } else if source_field.data_type().equals_datatype(target_type) { + // Field ID matches and either: + // - No name mapping present (trust the field ID) + // - Names also match (correct field, use it) Some(ColumnSource::PassThrough { source_index: *source_index, }) @@ -561,10 +560,7 @@ impl RecordBatchTransformer { source_index: *source_index, }) } - } - } else { - None - }; + }); if let Some(source) = field_by_id { source @@ -1029,7 +1025,7 @@ mod test { )])) } - /// Test for add_files partition column handling with field ID conflicts. + /// Test for add_files with Parquet files that have NO field IDs (Hive tables). /// /// This reproduces the scenario from Iceberg spec where: /// - Hive-style partitioned Parquet files are imported via add_files procedure @@ -1047,7 +1043,7 @@ mod test { /// 3. dept="hr" (from initial_default) - spec rule #3 /// 4. subdept="communications" (from Parquet via name mapping) - spec rule #2 #[test] - fn add_files_partition_columns_with_field_id_conflict() { + fn add_files_partition_columns_without_field_ids() { // Iceberg schema after add_files: id (partition), name, dept (partition), subdept let snapshot_schema = Arc::new( Schema::builder() @@ -1143,6 +1139,139 @@ mod test { assert_eq!(subdept_column.value(0), "communications"); } + /// Test for TRUE field ID conflicts where Parquet field IDs don't match Iceberg semantics. + /// + /// This is the critical test that exercises the field ID conflict detection code (lines 545-549). + /// + /// # Scenario + /// + /// Parquet file was written with its own field ID assignment: + /// - field_id=1 → "name" (String) + /// - field_id=2 → "salary" (Int) + /// + /// Then imported to Iceberg via add_files with NEW field IDs: + /// - field_id=1 → "id" (Int, partition column with initial_default=1) + /// - field_id=2 → "name" (String) + /// - field_id=3 → "dept" (String, partition column with initial_default="hr") + /// - field_id=4 → "salary" (Int) + /// + /// # The Conflict + /// + /// When looking for Iceberg field_id=2 ("name"): + /// - Parquet HAS a field_id=2, but it's "salary", not "name" + /// - This is a field ID conflict - same ID, different field + /// + /// # Expected Behavior Per Spec + /// + /// Per the spec, field_id=2 "name" is NOT PRESENT in the Parquet file (even though + /// a different field_id=2 exists). The implementation must: + /// 1. Detect the conflict (field ID matches, but names don't match) + /// 2. Treat Iceberg field_id=2 as "not present" + /// 3. Fall through to name mapping (spec rule #2) to find "name" by column name + /// + /// # Implementation + /// + /// This tests the code at lines 545-549: + /// ```rust + /// if name_mapping.is_some() && !name_matches { + /// // Field ID conflict detected + /// None + /// } + /// ``` + /// + /// # References + /// - Iceberg spec: https://iceberg.apache.org/spec/#column-projection + #[test] + fn add_files_with_true_field_id_conflict() { + use crate::spec::{MappedField, NameMapping}; + + // Iceberg schema after add_files import + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::optional(1, "id", Type::Primitive(PrimitiveType::Int)) + .with_initial_default(Literal::int(1)) + .into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "dept", Type::Primitive(PrimitiveType::String)) + .with_initial_default(Literal::string("hr")) + .into(), + NestedField::optional(4, "salary", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(), + ); + + // Parquet file schema with CONFLICTING field IDs: + // - field_id=1 is "name" in Parquet, but "id" in Iceberg (conflict!) + // - field_id=2 is "salary" in Parquet, but "name" in Iceberg (conflict!) + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("name", DataType::Utf8, true, "1"), + simple_field("salary", DataType::Int32, false, "2"), + ])); + + // Name mapping is CRITICAL - without it, we'd incorrectly use the conflicting field IDs + let name_mapping = Arc::new(NameMapping::new(vec![ + MappedField::new(Some(2), vec!["name".to_string()], vec![]), + MappedField::new(Some(4), vec!["salary".to_string()], vec![]), + ])); + + let projected_field_ids = [1, 2, 3, 4]; // id, name, dept, salary + + let mut transformer = RecordBatchTransformer::build_with_partition_data( + snapshot_schema, + &projected_field_ids, + None, + None, + Some(name_mapping), + ); + + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(StringArray::from(vec!["Alice"])), + Arc::new(Int32Array::from(vec![50000])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + // Verify the transformed RecordBatch correctly resolved field ID conflicts: + assert_eq!(result.num_columns(), 4); + assert_eq!(result.num_rows(), 1); + + // Column 0: id=1 (from initial_default, NOT from Parquet field_id=1 which is "name") + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_column.value(0), 1); + + // Column 1: name="Alice" (from Parquet via name mapping, NOT from field_id=2 which is "salary") + let name_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "Alice"); + + // Column 2: dept="hr" (from initial_default, not in Parquet file) + let dept_column = result + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(dept_column.value(0), "hr"); + + // Column 3: salary=50000 (from Parquet via name mapping, NOT from field_id=2 directly) + let salary_column = result + .column(3) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(salary_column.value(0), 50000); + } + /// Test for bucket partitioning where source columns must be read from data files. /// /// This test verifies correct implementation of the Iceberg spec's "Column Projection" rules: From b30195c791e9924ad13f21182ca07a3d93844342 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 4 Nov 2025 12:03:38 -0500 Subject: [PATCH 09/13] remove test accidentally brought in from #1778. --- .../src/arrow/caching_delete_file_loader.rs | 116 ------------------ .../src/arrow/record_batch_transformer.rs | 8 +- 2 files changed, 4 insertions(+), 120 deletions(-) diff --git a/crates/iceberg/src/arrow/caching_delete_file_loader.rs b/crates/iceberg/src/arrow/caching_delete_file_loader.rs index f1c4f86f23..078635c9be 100644 --- a/crates/iceberg/src/arrow/caching_delete_file_loader.rs +++ b/crates/iceberg/src/arrow/caching_delete_file_loader.rs @@ -788,120 +788,4 @@ mod tests { assert_eq!(data_col.value(1), "d"); assert_eq!(data_col.value(2), "g"); } - - /// Test loading a FileScanTask with BOTH positional and equality deletes. - /// Verifies the fix for the inverted condition that caused "Missing predicate for equality delete file" errors. - #[tokio::test] - async fn test_load_deletes_with_mixed_types() { - use crate::scan::FileScanTask; - use crate::spec::{DataFileFormat, Schema}; - - let tmp_dir = TempDir::new().unwrap(); - let table_location = tmp_dir.path(); - let file_io = FileIO::from_path(table_location.as_os_str().to_str().unwrap()) - .unwrap() - .build() - .unwrap(); - - // Create the data file schema - let data_file_schema = Arc::new( - Schema::builder() - .with_fields(vec![ - crate::spec::NestedField::optional( - 2, - "y", - crate::spec::Type::Primitive(crate::spec::PrimitiveType::Long), - ) - .into(), - crate::spec::NestedField::optional( - 3, - "z", - crate::spec::Type::Primitive(crate::spec::PrimitiveType::Long), - ) - .into(), - ]) - .build() - .unwrap(), - ); - - // Write positional delete file - let positional_delete_schema = crate::arrow::delete_filter::tests::create_pos_del_schema(); - let file_path_values = - vec![format!("{}/data-1.parquet", table_location.to_str().unwrap()); 4]; - let file_path_col = Arc::new(StringArray::from_iter_values(&file_path_values)); - let pos_col = Arc::new(Int64Array::from_iter_values(vec![0i64, 1, 2, 3])); - - let positional_deletes_to_write = - RecordBatch::try_new(positional_delete_schema.clone(), vec![ - file_path_col, - pos_col, - ]) - .unwrap(); - - let props = WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .build(); - - let pos_del_path = format!("{}/pos-del-mixed.parquet", table_location.to_str().unwrap()); - let file = File::create(&pos_del_path).unwrap(); - let mut writer = ArrowWriter::try_new( - file, - positional_deletes_to_write.schema(), - Some(props.clone()), - ) - .unwrap(); - writer.write(&positional_deletes_to_write).unwrap(); - writer.close().unwrap(); - - // Write equality delete file - let eq_delete_path = setup_write_equality_delete_file_1(table_location.to_str().unwrap()); - - // Create FileScanTask with BOTH positional and equality deletes - let pos_del = FileScanTaskDeleteFile { - file_path: pos_del_path, - file_type: DataContentType::PositionDeletes, - partition_spec_id: 0, - equality_ids: None, - }; - - let eq_del = FileScanTaskDeleteFile { - file_path: eq_delete_path.clone(), - file_type: DataContentType::EqualityDeletes, - partition_spec_id: 0, - equality_ids: Some(vec![2, 3]), // Only use field IDs that exist in both schemas - }; - - let file_scan_task = FileScanTask { - start: 0, - length: 0, - record_count: None, - data_file_path: format!("{}/data-1.parquet", table_location.to_str().unwrap()), - data_file_format: DataFileFormat::Parquet, - schema: data_file_schema.clone(), - project_field_ids: vec![2, 3], - predicate: None, - deletes: vec![pos_del, eq_del], - partition: None, - partition_spec: None, - name_mapping: None, - }; - - // Load the deletes - should handle both types without error - let delete_file_loader = CachingDeleteFileLoader::new(file_io.clone(), 10); - let delete_filter = delete_file_loader - .load_deletes(&file_scan_task.deletes, file_scan_task.schema_ref()) - .await - .unwrap() - .unwrap(); - - // Verify both delete types can be processed together - let result = delete_filter - .build_equality_delete_predicate(&file_scan_task) - .await; - assert!( - result.is_ok(), - "Failed to build equality delete predicate: {:?}", - result.err() - ); - } } diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index e586b1aa02..d3905cf0cb 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -537,9 +537,8 @@ impl RecordBatchTransformer { // - When name mapping is present, field ID matches alone aren't sufficient // - We verify the field NAME also matches to ensure it's the correct field // - If names don't match, we treat the field as "not present" and use name mapping - let field_by_id = field_id_to_source_schema_map - .get(field_id) - .and_then(|(source_field, source_index)| { + let field_by_id = field_id_to_source_schema_map.get(field_id).and_then( + |(source_field, source_index)| { let name_matches = source_field.name() == &iceberg_field.name; if name_mapping.is_some() && !name_matches { @@ -560,7 +559,8 @@ impl RecordBatchTransformer { source_index: *source_index, }) } - }); + }, + ); if let Some(source) = field_by_id { source From 135f1499d58cd8b3acda95d7535b8f147ceb0e6f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 4 Nov 2025 12:17:12 -0500 Subject: [PATCH 10/13] remove inadvertent changes, make comments more succinct --- crates/iceberg/src/arrow/reader.rs | 21 +-- .../src/arrow/record_batch_transformer.rs | 122 +++--------------- crates/iceberg/src/scan/task.rs | 7 - 3 files changed, 27 insertions(+), 123 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 02f60bdf8a..770566f93d 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -235,8 +235,6 @@ impl ArrowReader { // RecordBatchTransformer performs any transformations required on the RecordBatches // that come back from the file, such as type promotion, default column insertion // and column re-ordering. - // Always use build_with_partition_data to ensure name_mapping is passed through, - // even when partition spec/data aren't available. let mut record_batch_transformer = RecordBatchTransformer::build_with_partition_data( task.schema_ref(), task.project_field_ids(), @@ -1949,7 +1947,7 @@ message schema { start: 0, length: 0, record_count: None, - data_file_path: format!("{}/1.parquet", table_location), + data_file_path: format!("{table_location}/1.parquet"), data_file_format: DataFileFormat::Parquet, schema: schema.clone(), project_field_ids: vec![1], @@ -2323,21 +2321,16 @@ message schema { .unwrap(); let total_rows_task2: usize = result2.iter().map(|b| b.num_rows()).sum(); - println!( - "Task 2 (bytes {}-{}) returned {} rows", - rg1_start, file_end, total_rows_task2 - ); + println!("Task 2 (bytes {rg1_start}-{file_end}) returned {total_rows_task2} rows"); assert_eq!( total_rows_task1, 100, - "Task 1 should read only the first row group (100 rows), but got {} rows", - total_rows_task1 + "Task 1 should read only the first row group (100 rows), but got {total_rows_task1} rows" ); assert_eq!( total_rows_task2, 200, - "Task 2 should read only the second+third row groups (200 rows), but got {} rows", - total_rows_task2 + "Task 2 should read only the second+third row groups (200 rows), but got {total_rows_task2} rows" ); // Verify the actual data values are correct (not just the row count) @@ -2348,7 +2341,7 @@ message schema { .as_primitive::(); let first_val = id_col.value(0); let last_val = id_col.value(id_col.len() - 1); - println!("Task 1 data range: {} to {}", first_val, last_val); + println!("Task 1 data range: {first_val} to {last_val}"); assert_eq!(first_val, 0, "Task 1 should start with id=0"); assert_eq!(last_val, 99, "Task 1 should end with id=99"); @@ -2360,7 +2353,7 @@ message schema { .column(0) .as_primitive::(); let first_val = id_col.value(0); - println!("Task 2 first value: {}", first_val); + println!("Task 2 first value: {first_val}"); assert_eq!(first_val, 100, "Task 2 should start with id=100, not id=0"); } @@ -2418,7 +2411,7 @@ message schema { start: 0, length: 0, record_count: None, - data_file_path: format!("{}/old_file.parquet", table_location), + data_file_path: format!("{table_location}/old_file.parquet"), data_file_format: DataFileFormat::Parquet, schema: new_schema.clone(), project_field_ids: vec![1, 2], // Request both columns 'a' and 'b' diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index d3905cf0cb..fa85d49666 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -37,56 +37,18 @@ use crate::{Error, ErrorKind, Result}; /// Build a map of field ID to constant value for identity-partitioned fields. /// -/// This implements the Iceberg spec's "Column Projection" rule #1 -/// (https://iceberg.apache.org/spec/#column-projection): -/// > "Return the value from partition metadata if an Identity Transform exists for the field -/// > and the partition value is present in the `partition` struct on `data_file` object -/// > in the manifest." +/// Implements Iceberg spec "Column Projection" rule #1: use partition metadata constants +/// only for identity-transformed fields. Non-identity transforms (bucket, truncate, year, etc.) +/// store derived values in partition metadata, so source columns must be read from data files. /// -/// This matches Java's `PartitionUtil.constantsMap()` which only adds fields where: -/// ```java -/// if (field.transform().isIdentity()) { -/// idToConstant.put(field.sourceId(), converted); -/// } -/// ``` +/// Example: For `bucket(4, id)`, partition metadata has `id_bucket = 2` (bucket number), +/// but the actual `id` values (100, 200, 300) are only in the data file. /// -/// # Why only identity transforms? -/// -/// Non-identity transforms (bucket, truncate, year, month, day, hour) produce DERIVED values -/// that differ from the source column values. For example: -/// - `bucket(4, id)` produces hash values 0-3, not the actual `id` values -/// - `day(timestamp)` produces day-since-epoch integers, not the timestamp values -/// -/// These source columns MUST be read from the data file because partition metadata only -/// stores the transformed values (e.g., bucket number), not the original column values. -/// -/// # Java Implementation Reference -/// -/// This matches Java's `PartitionUtil.constantsMap()` (util/PartitionUtil.java): -/// ```java -/// public static Map constantsMap(PartitionData data, PartitionSpec spec) { -/// Map idToConstant = Maps.newHashMap(); -/// for (int pos = 0; pos < spec.fields().size(); pos += 1) { -/// PartitionField field = spec.fields().get(pos); -/// if (field.transform().isIdentity()) { // <-- ONLY identity transforms -/// Object converted = convertConstant(field.sourceId(), data.get(pos, javaClass)); -/// idToConstant.put(field.sourceId(), converted); -/// } -/// } -/// return idToConstant; -/// } -/// ``` -/// -/// # Example: Bucket Partitioning -/// -/// For a table partitioned by `bucket(4, id)`: -/// - Partition metadata stores: `id_bucket = 2` (the bucket number) -/// - Data file contains: `id = 100, 200, 300` (the actual values) -/// - Reading must use data from the file, not the constant `2` from partition metadata +/// Matches Java's `PartitionUtil.constantsMap()` which filters `if (field.transform().isIdentity())`. /// /// # References -/// - Iceberg spec: format/spec.md "Column Projection" section -/// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java:constantsMap() +/// - Spec: https://iceberg.apache.org/spec/#column-projection +/// - Java: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java:constantsMap() fn constants_map( partition_spec: &PartitionSpec, partition_data: &Struct, @@ -191,8 +153,6 @@ pub(crate) struct RecordBatchTransformer { partition_data: Option, // Optional name mapping for resolving field IDs from column names - // Per Iceberg spec rule #2: "Use schema.name-mapping.default metadata - // to map field id to columns without field id" name_mapping: Option>, // BatchTransform gets lazily constructed based on the schema of @@ -218,70 +178,28 @@ impl RecordBatchTransformer { /// Build a RecordBatchTransformer with partition spec and data for proper constant identification. /// - /// # Overview - /// - /// This method implements the Iceberg spec's "Column Projection" rules - /// (https://iceberg.apache.org/spec/#column-projection) for resolving field IDs that are - /// "not present" in a data file: - /// + /// Implements the Iceberg spec's "Column Projection" rules for resolving field IDs "not present" in data files: /// 1. Return the value from partition metadata if an Identity Transform exists /// 2. Use schema.name-mapping.default metadata to map field id to columns without field id /// 3. Return the default value if it has a defined initial-default /// 4. Return null in all other cases /// - /// # Why this method was added - /// - /// The gap in iceberg-rust was that `FileScanTask` had no way to pass partition information - /// to `RecordBatchTransformer`. This caused two problems: - /// - /// 1. **Incorrect handling of bucket partitioning**: Without partition spec information, - /// iceberg-rust couldn't distinguish between: - /// - Identity transforms (use constants from partition metadata) - /// - Non-identity transforms like bucket (read from data file) + /// # Why this method exists /// - /// This caused bucket-partitioned source columns to be incorrectly treated as constants, - /// breaking runtime filtering and returning incorrect query results. + /// 1. **Bucket partitioning**: Distinguish identity transforms (use partition metadata constants) + /// from non-identity transforms like bucket (read from data file) to enable runtime filtering on + /// bucket-partitioned columns. /// - /// 2. **Add_files field ID conflicts**: When importing Hive tables via add_files, - /// partition columns with `initial_default` values could have field IDs that conflicted - /// with data column field IDs in the Parquet file. + /// 2. **Add_files field ID conflicts**: When importing Hive tables, partition columns can have field IDs + /// conflicting with Parquet data columns (e.g., Parquet has field_id=1→"name", but Iceberg expects + /// field_id=1→"id"). Per spec, such fields are "not present" and should use name mapping (rule #2). /// - /// Example: - /// - Parquet file written with: field_id=1→"name", field_id=2→"dept" - /// - Imported via add_files: field_id=1→"id" (partition), field_id=2→"name", field_id=3→"dept" - /// - /// When looking for field_id=1 ("id"), we find field_id=1 in the Parquet file, but it's - /// the WRONG field (it's "name"). Per the spec, the correct field (id=1, name="id") is - /// "not present" in the file and should be resolved via name mapping (rule #2) or - /// initial-default (rule #3). - /// - /// # The fix - /// - /// This method accepts `partition_spec`, `partition_data`, and `name_mapping`, which are used to: - /// - Build a `constants_map` that ONLY includes identity-transformed partition fields - /// (matching Java's `PartitionUtil.constantsMap()` behavior) - /// - Detect field ID conflicts by verifying both field ID AND name match (when name mapping present) - /// - Apply name mapping when field IDs are missing or conflicting (spec rule #2) - /// - /// This matches Java's approach (ParquetSchemaUtil.applyNameMapping, ReadConf.java lines 83-85) - /// which rewrites Parquet schema field IDs based on names before projection. Our implementation - /// detects conflicts during projection but achieves the same result. - /// - /// # What was changed - /// - /// To enable this fix, the following fields were added to `FileScanTask`: - /// - `partition: Option` - The partition data for this file - /// - `partition_spec: Option>` - The actual partition spec - /// - `name_mapping: Option>` - The name mapping from table metadata - /// - /// These fields should be populated by any system that reads Iceberg tables and provides - /// FileScanTasks to the ArrowReader. + /// This matches Java's ParquetSchemaUtil.applyNameMapping approach but detects conflicts during projection. /// /// # References - /// - Iceberg spec: https://iceberg.apache.org/spec/#column-projection - /// - Java impl: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java - /// - Java impl: parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java - /// - Java test: spark/src/test/java/.../TestRuntimeFiltering.java + /// - Spec: https://iceberg.apache.org/spec/#column-projection + /// - Java: core/src/main/java/org/apache/iceberg/util/PartitionUtil.java + /// - Java: parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java pub(crate) fn build_with_partition_data( snapshot_schema: Arc, projected_iceberg_field_ids: &[i32], diff --git a/crates/iceberg/src/scan/task.rs b/crates/iceberg/src/scan/task.rs index 04259bd34c..e1ef241a57 100644 --- a/crates/iceberg/src/scan/task.rs +++ b/crates/iceberg/src/scan/task.rs @@ -99,13 +99,6 @@ pub struct FileScanTask { /// Name mapping from table metadata (property: schema.name-mapping.default), /// used to resolve field IDs from column names when Parquet files lack field IDs /// or have field ID conflicts. - /// - /// Per Iceberg spec rule #2: "Use schema.name-mapping.default metadata to map - /// field id to columns without field id". - /// - /// This is essential for scenarios like: - /// - Hive table migrations via add_files where Parquet has no field IDs - /// - Field ID conflicts where partition columns conflict with data column IDs #[serde(default)] #[serde(skip_serializing_if = "Option::is_none")] #[serde(serialize_with = "serialize_not_implemented")] From 37b1513e62a36b6d195bacf43bb76a36cdc5c0ab Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 5 Nov 2025 07:00:48 -0500 Subject: [PATCH 11/13] Fix test after merging in main. --- crates/iceberg/src/arrow/caching_delete_file_loader.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/crates/iceberg/src/arrow/caching_delete_file_loader.rs b/crates/iceberg/src/arrow/caching_delete_file_loader.rs index 8a3ab3a955..f1c4f86f23 100644 --- a/crates/iceberg/src/arrow/caching_delete_file_loader.rs +++ b/crates/iceberg/src/arrow/caching_delete_file_loader.rs @@ -881,6 +881,9 @@ mod tests { project_field_ids: vec![2, 3], predicate: None, deletes: vec![pos_del, eq_del], + partition: None, + partition_spec: None, + name_mapping: None, }; // Load the deletes - should handle both types without error From c2fcc6646a209740e33988723f0045ecd4c46d3a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 6 Nov 2025 16:54:43 -0500 Subject: [PATCH 12/13] Address PR feedback. --- .../src/arrow/record_batch_transformer.rs | 457 ++++++++---------- 1 file changed, 194 insertions(+), 263 deletions(-) diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index fa85d49666..766d9d0ea1 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -191,8 +191,8 @@ impl RecordBatchTransformer { /// bucket-partitioned columns. /// /// 2. **Add_files field ID conflicts**: When importing Hive tables, partition columns can have field IDs - /// conflicting with Parquet data columns (e.g., Parquet has field_id=1→"name", but Iceberg expects - /// field_id=1→"id"). Per spec, such fields are "not present" and should use name mapping (rule #2). + /// conflicting with Parquet data columns (e.g., Parquet has field_id=1->"name", but Iceberg expects + /// field_id=1->"id"). Per spec, such fields are "not present" and should use name mapping (rule #2). /// /// This matches Java's ParquetSchemaUtil.applyNameMapping approach but detects conflicts during projection. /// @@ -400,146 +400,91 @@ impl RecordBatchTransformer { "Field not found in snapshot schema", ))?; - // Determine how to source this column per Iceberg spec "Column Projection" rules. + // Iceberg spec's "Column Projection" rules (https://iceberg.apache.org/spec/#column-projection). + // For fields "not present" in data files: + // 1. Use partition metadata (identity transforms only) + // 2. Use name mapping + // 3. Use initial_default + // 4. Return null // - // Per the spec (https://iceberg.apache.org/spec/#column-projection): - // "Values for field ids which are not present in a data file must be resolved - // according the following rules:" - // - // 1. "Return the value from partition metadata if an Identity Transform exists - // for the field and the partition value is present in the partition struct - // on data_file object in the manifest." - // 2. "Use schema.name-mapping.default metadata to map field id to columns - // without field id as described below and use the column if it is present." - // 3. "Return the default value if it has a defined initial-default" - // 4. "Return null in all other cases" + // WHY check partition constants before Parquet field IDs (Java: BaseParquetReaders.java:299): + // In add_files scenarios, partition columns may exist in BOTH Parquet AND partition metadata. + // Partition metadata is authoritative - it defines which partition this file belongs to. + + // WHY verify names when checking field IDs: + // add_files can create field ID conflicts (Parquet field_id=1->"name", Iceberg field_id=1->"id"). + // Name mismatches with name_mapping present indicate conflicts, treat field as "not present". + // Without name_mapping, name mismatches are just column renames, so trust the field ID. + // (Java: ParquetSchemaUtil.applyNameMapping, TestAddFilesProcedure.addDataPartitioned) + let field_by_id = field_id_to_source_schema_map.get(field_id).and_then( + |(source_field, source_index)| { + if name_mapping.is_some() { + let name_matches = source_field.name() == &iceberg_field.name; + if !name_matches { + return None; // Field ID conflict, treat as "not present" + } + } + if source_field.data_type().equals_datatype(target_type) { + Some(ColumnSource::PassThrough { + source_index: *source_index, + }) + } else { + Some(ColumnSource::Promote { + target_type: target_type.clone(), + source_index: *source_index, + }) + } + }, + ); + + // Apply spec's fallback steps for "not present" fields. let column_source = if let Some(constant_value) = constants_map.get(field_id) { - // Spec rule #1: Identity-partitioned column - use constant from partition metadata + // Rule #1: Identity partition constant ColumnSource::Add { value: Some(constant_value.clone()), target_type: target_type.clone(), } + } else if let Some(source) = field_by_id { + source } else { - // Check if field ID exists in Parquet, but verify it's the CORRECT field. - // - // Per the Iceberg spec (https://iceberg.apache.org/spec/#column-projection): - // "Values for field ids which are NOT PRESENT in a data file must be resolved - // according the following rules..." - // - // The key insight: In add_files scenarios (Hive table migrations), Parquet files - // may have field IDs that conflict with the Iceberg schema's field IDs: - // - // Example: - // - Parquet file written with: field_id=1→"name", field_id=2→"dept" - // - Imported via add_files with: field_id=1→"id" (partition), field_id=2→"name", field_id=3→"dept" - // - // When we look for Iceberg field_id=1 ("id"), we find a field_id=1 in the Parquet file, - // but it's the WRONG field (it's "name", not "id"). The correct field (id=1, name="id") - // is NOT PRESENT in the Parquet file - it only exists as partition metadata. - // - // Per the spec: when a field is "not present", we should apply rules 2-4 (name mapping, - // initial-default, or null). - // - // Java's approach (ParquetSchemaUtil.applyNameMapping, ReadConf.java lines 83-85): - // Java REWRITES the Parquet schema's field IDs based on names BEFORE projection: - // ```java - // if (nameMapping != null) { - // typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); - // // Now field IDs match Iceberg schema based on name mapping - // this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); - // } - // ``` - // - // Our approach achieves the same result but detects conflicts DURING projection: - // - When name mapping is present, field ID matches alone aren't sufficient - // - We verify the field NAME also matches to ensure it's the correct field - // - If names don't match, we treat the field as "not present" and use name mapping - let field_by_id = field_id_to_source_schema_map.get(field_id).and_then( - |(source_field, source_index)| { - let name_matches = source_field.name() == &iceberg_field.name; - - if name_mapping.is_some() && !name_matches { - // Field ID conflict detected: Parquet has this field ID but for a different field. - // The field we're looking for (this field_id + this name) is NOT PRESENT in the file. - // Per spec: treat as "not present" and fall through to name mapping (rule #2). - None - } else if source_field.data_type().equals_datatype(target_type) { - // Field ID matches and either: - // - No name mapping present (trust the field ID) - // - Names also match (correct field, use it) - Some(ColumnSource::PassThrough { - source_index: *source_index, + // Rule #2: Name mapping (Java: ReadConf.java:83-85) + let name_mapped_column = name_mapping.and_then(|mapping| { + mapping.fields().iter().find_map(|mapped_field| { + if mapped_field.field_id() == Some(*field_id) { + mapped_field.names().iter().find_map(|name| { + field_name_to_source_schema_map + .get(name) + .map(|(field, idx)| (field.clone(), *idx)) }) } else { - Some(ColumnSource::Promote { - target_type: target_type.clone(), - source_index: *source_index, - }) + None } - }, - ); + }) + }); - if let Some(source) = field_by_id { - source + if let Some((source_field, source_index)) = name_mapped_column { + if source_field.data_type().equals_datatype(target_type) { + ColumnSource::PassThrough { source_index } + } else { + ColumnSource::Promote { + target_type: target_type.clone(), + source_index, + } + } } else { - // Field not found by ID, or field ID conflict detected. - // Per spec: field is "not present", apply remaining rules. - // Field NOT in Parquet by field ID - try spec rule #2 (name mapping) - // - // This handles scenarios like Hive table migrations where Parquet files may: - // - Have no field IDs at all - // - Have conflicting field IDs (e.g., add_files with partition columns) - // - // Per Java's implementation (ReadConf.java lines 83-85): - // ```java - // } else if (nameMapping != null) { - // typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); - // this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); - // ``` - // - // The name mapping provides a fallback: "this field ID corresponds to these possible names" - let name_mapped_column = name_mapping.and_then(|mapping| { - // Find the mapped field for this field ID - // The NameMapping structure allows looking up by field ID to get the names - mapping.fields().iter().find_map(|mapped_field| { - if mapped_field.field_id() == Some(*field_id) { - // Try each possible name for this field - mapped_field.names().iter().find_map(|name| { - field_name_to_source_schema_map - .get(name) - .map(|(field, idx)| (field.clone(), *idx)) - }) + // Rules #3 and #4: initial_default or null + let default_value = + iceberg_field.initial_default.as_ref().and_then(|lit| { + if let Literal::Primitive(prim) = lit { + Some(prim.clone()) } else { None } - }) - }); - - if let Some((source_field, source_index)) = name_mapped_column { - // Spec rule #2: Found column via name mapping - read it from file - if source_field.data_type().equals_datatype(target_type) { - ColumnSource::PassThrough { source_index } - } else { - ColumnSource::Promote { - target_type: target_type.clone(), - source_index, - } - } - } else { - // Spec rules #3 and #4: Use initial_default if present, otherwise null - let default_value = - iceberg_field.initial_default.as_ref().and_then(|lit| { - if let Literal::Primitive(prim) = lit { - Some(prim.clone()) - } else { - None - } - }); - ColumnSource::Add { - value: default_value, - target_type: target_type.clone(), - } + }); + ColumnSource::Add { + value: default_value, + target_type: target_type.clone(), } } }; @@ -1057,139 +1002,6 @@ mod test { assert_eq!(subdept_column.value(0), "communications"); } - /// Test for TRUE field ID conflicts where Parquet field IDs don't match Iceberg semantics. - /// - /// This is the critical test that exercises the field ID conflict detection code (lines 545-549). - /// - /// # Scenario - /// - /// Parquet file was written with its own field ID assignment: - /// - field_id=1 → "name" (String) - /// - field_id=2 → "salary" (Int) - /// - /// Then imported to Iceberg via add_files with NEW field IDs: - /// - field_id=1 → "id" (Int, partition column with initial_default=1) - /// - field_id=2 → "name" (String) - /// - field_id=3 → "dept" (String, partition column with initial_default="hr") - /// - field_id=4 → "salary" (Int) - /// - /// # The Conflict - /// - /// When looking for Iceberg field_id=2 ("name"): - /// - Parquet HAS a field_id=2, but it's "salary", not "name" - /// - This is a field ID conflict - same ID, different field - /// - /// # Expected Behavior Per Spec - /// - /// Per the spec, field_id=2 "name" is NOT PRESENT in the Parquet file (even though - /// a different field_id=2 exists). The implementation must: - /// 1. Detect the conflict (field ID matches, but names don't match) - /// 2. Treat Iceberg field_id=2 as "not present" - /// 3. Fall through to name mapping (spec rule #2) to find "name" by column name - /// - /// # Implementation - /// - /// This tests the code at lines 545-549: - /// ```rust - /// if name_mapping.is_some() && !name_matches { - /// // Field ID conflict detected - /// None - /// } - /// ``` - /// - /// # References - /// - Iceberg spec: https://iceberg.apache.org/spec/#column-projection - #[test] - fn add_files_with_true_field_id_conflict() { - use crate::spec::{MappedField, NameMapping}; - - // Iceberg schema after add_files import - let snapshot_schema = Arc::new( - Schema::builder() - .with_schema_id(0) - .with_fields(vec![ - NestedField::optional(1, "id", Type::Primitive(PrimitiveType::Int)) - .with_initial_default(Literal::int(1)) - .into(), - NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), - NestedField::optional(3, "dept", Type::Primitive(PrimitiveType::String)) - .with_initial_default(Literal::string("hr")) - .into(), - NestedField::optional(4, "salary", Type::Primitive(PrimitiveType::Int)).into(), - ]) - .build() - .unwrap(), - ); - - // Parquet file schema with CONFLICTING field IDs: - // - field_id=1 is "name" in Parquet, but "id" in Iceberg (conflict!) - // - field_id=2 is "salary" in Parquet, but "name" in Iceberg (conflict!) - let parquet_schema = Arc::new(ArrowSchema::new(vec![ - simple_field("name", DataType::Utf8, true, "1"), - simple_field("salary", DataType::Int32, false, "2"), - ])); - - // Name mapping is CRITICAL - without it, we'd incorrectly use the conflicting field IDs - let name_mapping = Arc::new(NameMapping::new(vec![ - MappedField::new(Some(2), vec!["name".to_string()], vec![]), - MappedField::new(Some(4), vec!["salary".to_string()], vec![]), - ])); - - let projected_field_ids = [1, 2, 3, 4]; // id, name, dept, salary - - let mut transformer = RecordBatchTransformer::build_with_partition_data( - snapshot_schema, - &projected_field_ids, - None, - None, - Some(name_mapping), - ); - - let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ - Arc::new(StringArray::from(vec!["Alice"])), - Arc::new(Int32Array::from(vec![50000])), - ]) - .unwrap(); - - let result = transformer.process_record_batch(parquet_batch).unwrap(); - - // Verify the transformed RecordBatch correctly resolved field ID conflicts: - assert_eq!(result.num_columns(), 4); - assert_eq!(result.num_rows(), 1); - - // Column 0: id=1 (from initial_default, NOT from Parquet field_id=1 which is "name") - let id_column = result - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(id_column.value(0), 1); - - // Column 1: name="Alice" (from Parquet via name mapping, NOT from field_id=2 which is "salary") - let name_column = result - .column(1) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(name_column.value(0), "Alice"); - - // Column 2: dept="hr" (from initial_default, not in Parquet file) - let dept_column = result - .column(2) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(dept_column.value(0), "hr"); - - // Column 3: salary=50000 (from Parquet via name mapping, NOT from field_id=2 directly) - let salary_column = result - .column(3) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(salary_column.value(0), 50000); - } - /// Test for bucket partitioning where source columns must be read from data files. /// /// This test verifies correct implementation of the Iceberg spec's "Column Projection" rules: @@ -1558,10 +1370,10 @@ mod test { /// according the following rules:" /// /// This test creates a scenario where each rule is exercised: - /// - Rule #1: dept (identity-partitioned) → constant from partition metadata - /// - Rule #2: data (via name mapping) → read from Parquet file by name - /// - Rule #3: category (initial_default) → use default value - /// - Rule #4: notes (no default) → return null + /// - Rule #1: dept (identity-partitioned) -> constant from partition metadata + /// - Rule #2: data (via name mapping) -> read from Parquet file by name + /// - Rule #3: category (initial_default) -> use default value + /// - Rule #4: notes (no default) -> return null /// /// # References /// - Iceberg spec: format/spec.md "Column Projection" section @@ -1687,4 +1499,123 @@ mod test { assert!(notes_column.is_null(0)); assert!(notes_column.is_null(1)); } + + /// Verifies field ID conflict detection for add_files imports. + /// + /// WHY: add_files can import Parquet with conflicting field IDs (field_id=1->"name" in Parquet + /// vs field_id=1->"id" in Iceberg). Name-checking detects conflicts, treats fields as "not present", + /// allowing spec fallback to partition constants and name mapping. + /// + /// Reproduces: TestAddFilesProcedure.addDataPartitioned + #[test] + fn add_files_with_field_id_conflicts_like_java_test() { + use crate::spec::{MappedField, NameMapping, Struct, Transform}; + + // Iceberg schema (field IDs 1-4) + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(3, "dept", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(4, "subdept", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build() + .unwrap(), + ); + + let partition_spec = Arc::new( + crate::spec::PartitionSpec::builder(snapshot_schema.clone()) + .with_spec_id(0) + .add_partition_field("id", "id", Transform::Identity) + .unwrap() + .build() + .unwrap(), + ); + + let partition_data = Struct::from_iter(vec![Some(Literal::int(1))]); + + // Parquet schema with CONFLICTING field IDs (1-3 instead of 1-4) + let parquet_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("name", DataType::Utf8, false, "1"), + simple_field("dept", DataType::Utf8, false, "2"), + simple_field("subdept", DataType::Utf8, false, "3"), + ])); + + // WHY name mapping: Resolves conflicts by mapping Iceberg field IDs to Parquet column names + let name_mapping = Arc::new(NameMapping::new(vec![ + MappedField::new(Some(2), vec!["name".to_string()], vec![]), + MappedField::new(Some(3), vec!["dept".to_string()], vec![]), + MappedField::new(Some(4), vec!["subdept".to_string()], vec![]), + ])); + + let projected_field_ids = [1, 2, 3, 4]; // id, name, dept, subdept + + let mut transformer = RecordBatchTransformer::build_with_partition_data( + snapshot_schema, + &projected_field_ids, + Some(partition_spec), + Some(partition_data), + Some(name_mapping), + ); + + // Parquet data (one row) + let parquet_batch = RecordBatch::try_new(parquet_schema, vec![ + Arc::new(StringArray::from(vec!["John Doe"])), // name column + Arc::new(StringArray::from(vec!["Engineering"])), // dept column + Arc::new(StringArray::from(vec!["Backend"])), // subdept column + ]) + .unwrap(); + + let result = transformer.process_record_batch(parquet_batch).unwrap(); + + assert_eq!(result.num_columns(), 4); + assert_eq!(result.num_rows(), 1); + + // Verify: id from partition constant (conflict with Parquet field_id=1) + assert_eq!( + result + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0), + 1 + ); + + // Verify: name via name mapping (conflict with Parquet field_id=2) + assert_eq!( + result + .column(1) + .as_any() + .downcast_ref::() + .unwrap() + .value(0), + "John Doe" + ); + + // Verify: dept via name mapping (conflict with Parquet field_id=3) + assert_eq!( + result + .column(2) + .as_any() + .downcast_ref::() + .unwrap() + .value(0), + "Engineering" + ); + + // Verify: subdept via name mapping (not in Parquet by field ID) + assert_eq!( + result + .column(3) + .as_any() + .downcast_ref::() + .unwrap() + .value(0), + "Backend" + ); + } } From 668e78b6a3086ccbc98cc123046a7f8333301f70 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 6 Nov 2025 18:38:30 -0500 Subject: [PATCH 13/13] Adjust comments, mostly just to kick CI. --- crates/iceberg/src/arrow/record_batch_transformer.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index 766d9d0ea1..a4b22eb102 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -407,11 +407,11 @@ impl RecordBatchTransformer { // 3. Use initial_default // 4. Return null // - // WHY check partition constants before Parquet field IDs (Java: BaseParquetReaders.java:299): + // Why check partition constants before Parquet field IDs (Java: BaseParquetReaders.java:299): // In add_files scenarios, partition columns may exist in BOTH Parquet AND partition metadata. // Partition metadata is authoritative - it defines which partition this file belongs to. - // WHY verify names when checking field IDs: + // Why verify names when checking field IDs: // add_files can create field ID conflicts (Parquet field_id=1->"name", Iceberg field_id=1->"id"). // Name mismatches with name_mapping present indicate conflicts, treat field as "not present". // Without name_mapping, name mismatches are just column renames, so trust the field ID. @@ -1502,7 +1502,7 @@ mod test { /// Verifies field ID conflict detection for add_files imports. /// - /// WHY: add_files can import Parquet with conflicting field IDs (field_id=1->"name" in Parquet + /// Why: add_files can import Parquet with conflicting field IDs (field_id=1->"name" in Parquet /// vs field_id=1->"id" in Iceberg). Name-checking detects conflicts, treats fields as "not present", /// allowing spec fallback to partition constants and name mapping. ///