-
Notifications
You must be signed in to change notification settings - Fork 440
feat: Variant Support #2188
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
feat: Variant Support #2188
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -768,6 +768,10 @@ impl ArrowReader { | |
| Self::include_leaf_field_id(&map_type.key_field, field_ids); | ||
| Self::include_leaf_field_id(&map_type.value_field, field_ids); | ||
| } | ||
| // Variant is a leaf type for Parquet projection purposes (like a primitive). | ||
| Type::Variant(_) => { | ||
| field_ids.push(field.id); | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -840,8 +844,46 @@ impl ArrowReader { | |
| arrow_schema: &ArrowSchemaRef, | ||
| type_promotion_is_valid: fn(Option<&PrimitiveType>, Option<&PrimitiveType>) -> bool, | ||
| ) -> Result<ProjectionMask> { | ||
| let mut column_map = HashMap::new(); | ||
| // Maps field_id → leaf column indices. Vec because variant contributes two | ||
| // leaves (metadata + value) under a single field ID. | ||
| let mut column_map: HashMap<i32, Vec<usize>> = HashMap::new(); | ||
| let fields = arrow_schema.fields(); | ||
| // HashSet for O(1) membership checks instead of O(n) slice scans. | ||
| let leaf_field_id_set: HashSet<i32> = leaf_field_ids.iter().copied().collect(); | ||
|
|
||
| // Variant fields are an Iceberg leaf type but a Parquet GROUP. Their | ||
| // sub-fields (metadata, value) carry no embedded field IDs — only the | ||
| // parent group has the field ID. filter_leaves therefore never finds | ||
| // them via the standard field-ID scan below. | ||
| // | ||
| // Java's PruneColumns.variant() simply returns the group as-is with no | ||
| // type-compatibility check (isStruct() also short-circuits on isVariantType()). | ||
| // We replicate that here: pre-scan top-level Arrow struct fields whose | ||
| // field ID resolves to Type::Variant and record all their sub-fields so | ||
| // the second filter_leaves can include them directly. | ||
| let mut variant_sub_fields: HashMap<FieldRef, i32> = HashMap::new(); | ||
| for top_field in fields.iter() { | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What would happen if variant is nested within another type? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It will fail to find the leaf node metadata/value in the variant. The following are errors that can happen if nested in a Map. |
||
| let Some(field_id) = top_field | ||
| .metadata() | ||
| .get(PARQUET_FIELD_ID_META_KEY) | ||
| .and_then(|s| i32::from_str(s).ok()) | ||
| else { | ||
| continue; | ||
| }; | ||
| if !leaf_field_id_set.contains(&field_id) { | ||
| continue; | ||
| } | ||
| let Some(iceberg_field) = iceberg_schema_of_task.field_by_id(field_id) else { | ||
| continue; | ||
| }; | ||
| if let Type::Variant(_) = iceberg_field.field_type.as_ref() | ||
| && let DataType::Struct(sub_fields) = top_field.data_type() | ||
| { | ||
| for sub_field in sub_fields { | ||
| variant_sub_fields.insert(sub_field.clone(), field_id); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| // Pre-project only the fields that have been selected, possibly avoiding converting | ||
| // some Arrow types that are not yet supported. | ||
|
|
@@ -853,14 +895,22 @@ impl ArrowReader { | |
| .and_then(|field_id| i32::from_str(field_id).ok()) | ||
| .is_some_and(|field_id| { | ||
| projected_fields.insert((*f).clone(), field_id); | ||
| leaf_field_ids.contains(&field_id) | ||
| leaf_field_id_set.contains(&field_id) | ||
| }) | ||
| }), | ||
| arrow_schema.metadata().clone(), | ||
| ); | ||
| let iceberg_schema = arrow_schema_to_schema(&projected_arrow_schema)?; | ||
|
|
||
| fields.filter_leaves(|idx, field| { | ||
| // Variant sub-fields: parent group carries the field ID, not the leaf. | ||
| // Skip type-promotion check — Type::Variant is not a primitive type | ||
| // (matches Java's PruneColumns.variant() which returns the group unchanged). | ||
| if let Some(&variant_field_id) = variant_sub_fields.get(field) { | ||
| column_map.entry(variant_field_id).or_default().push(idx); | ||
| return true; | ||
| } | ||
|
|
||
| let Some(field_id) = projected_fields.get(field).cloned() else { | ||
| return false; | ||
| }; | ||
|
|
@@ -882,16 +932,16 @@ impl ArrowReader { | |
| return false; | ||
| } | ||
|
|
||
| column_map.insert(field_id, idx); | ||
| column_map.entry(field_id).or_default().push(idx); | ||
| true | ||
| }); | ||
|
|
||
| // Schema evolution: New columns may not exist in old Parquet files. | ||
| // We only project existing columns; RecordBatchTransformer adds default/NULL values. | ||
| let mut indices = vec![]; | ||
| for field_id in leaf_field_ids { | ||
| if let Some(col_idx) = column_map.get(field_id) { | ||
| indices.push(*col_idx); | ||
| if let Some(col_indices) = column_map.get(field_id) { | ||
| indices.extend_from_slice(col_indices); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -1980,7 +2030,8 @@ mod tests { | |
| use crate::io::FileIO; | ||
| use crate::scan::{FileScanTask, FileScanTaskDeleteFile, FileScanTaskStream}; | ||
| use crate::spec::{ | ||
| DataContentType, DataFileFormat, Datum, NestedField, PrimitiveType, Schema, SchemaRef, Type, | ||
| DataContentType, DataFileFormat, Datum, NestedField, PrimitiveType, Schema, SchemaRef, | ||
| Type, VariantType, | ||
| }; | ||
|
|
||
| fn table_schema_simple() -> SchemaRef { | ||
|
|
@@ -2148,6 +2199,100 @@ message schema { | |
| assert_eq!(mask, ProjectionMask::leaves(&parquet_schema, vec![0])); | ||
| } | ||
|
|
||
| /// Variant fields are an Iceberg leaf type but a Parquet GROUP whose sub-fields carry | ||
| /// no embedded field IDs. The projection mask must include both metadata and value | ||
| /// leaves when the variant field ID is requested, and must not drop the variant when | ||
| /// it is projected alongside ordinary primitive columns. | ||
| #[test] | ||
| fn test_arrow_projection_mask_variant() { | ||
| // Iceberg schema: c1 (String, id=1) + v (Variant, id=2) | ||
| let schema = Arc::new( | ||
| Schema::builder() | ||
| .with_schema_id(1) | ||
| .with_fields(vec![ | ||
| NestedField::required(1, "c1", Type::Primitive(PrimitiveType::String)).into(), | ||
| NestedField::required(2, "v", Type::Variant(VariantType)).into(), | ||
| ]) | ||
| .build() | ||
| .unwrap(), | ||
| ); | ||
|
|
||
| // Arrow schema: c1 with field ID 1; v as Struct(metadata: Binary, value: Binary) | ||
| // with field ID 2 on the struct but NO field IDs on the sub-fields — that is the | ||
| // Parquet variant wire format. | ||
| let arrow_schema = Arc::new(ArrowSchema::new(vec![ | ||
| Field::new("c1", DataType::Utf8, false).with_metadata(HashMap::from([( | ||
| PARQUET_FIELD_ID_META_KEY.to_string(), | ||
| "1".to_string(), | ||
| )])), | ||
| Field::new( | ||
| "v", | ||
| DataType::Struct(arrow_schema::Fields::from(vec![ | ||
| Arc::new(Field::new("metadata", DataType::Binary, false)), | ||
| Arc::new(Field::new("value", DataType::Binary, false)), | ||
| ])), | ||
| false, | ||
| ) | ||
| .with_metadata(HashMap::from([( | ||
| PARQUET_FIELD_ID_META_KEY.to_string(), | ||
| "2".to_string(), | ||
| )])), | ||
| ])); | ||
|
|
||
| // Parquet message: c1 is leaf 0; variant sub-fields metadata=leaf 1, value=leaf 2. | ||
| // No field IDs on sub-leaves — matching the real Iceberg/Spark-written variant format. | ||
| let message_type = " | ||
| message schema { | ||
| required binary c1 (STRING) = 1; | ||
| required group v = 2 { | ||
| required binary metadata; | ||
| required binary value; | ||
| } | ||
| } | ||
| "; | ||
| let parquet_type = parse_message_type(message_type).expect("should parse schema"); | ||
| let parquet_schema = SchemaDescriptor::new(Arc::new(parquet_type)); | ||
|
|
||
| // Both fields: all three leaves must be included. | ||
| let mask = ArrowReader::get_arrow_projection_mask( | ||
| &[1, 2], | ||
| &schema, | ||
| &parquet_schema, | ||
| &arrow_schema, | ||
| false, | ||
| ) | ||
| .expect("projection mask for c1 + v"); | ||
| assert_eq!(mask, ProjectionMask::leaves(&parquet_schema, vec![0, 1, 2])); | ||
|
|
||
| // Variant only: leaves 1 (metadata) and 2 (value) must be included. | ||
| let mask_variant_only = ArrowReader::get_arrow_projection_mask( | ||
| &[2], | ||
| &schema, | ||
| &parquet_schema, | ||
| &arrow_schema, | ||
| false, | ||
| ) | ||
| .expect("projection mask for v only"); | ||
| assert_eq!( | ||
| mask_variant_only, | ||
| ProjectionMask::leaves(&parquet_schema, vec![1, 2]), | ||
| ); | ||
|
|
||
| // Primitive only: leaf 0 (c1) must be included; variant NOT included. | ||
| let mask_primitive_only = ArrowReader::get_arrow_projection_mask( | ||
| &[1], | ||
| &schema, | ||
| &parquet_schema, | ||
| &arrow_schema, | ||
| false, | ||
| ) | ||
| .expect("projection mask for c1 only"); | ||
| assert_eq!( | ||
| mask_primitive_only, | ||
| ProjectionMask::leaves(&parquet_schema, vec![0]), | ||
| ); | ||
| } | ||
|
|
||
| #[tokio::test] | ||
| async fn test_kleene_logic_or_behaviour() { | ||
| // a IS NULL OR a = 'foo' | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we can just return "variant".to_string(), on glue it would look like below: