Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 7 additions & 0 deletions crates/catalog/glue/src/schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,13 @@ impl SchemaVisitor for GlueSchemaBuilder {

Ok(glue_type)
}

fn variant(&mut self, _v: &iceberg::spec::VariantType) -> Result<Self::T> {
Err(Error::new(
ErrorKind::FeatureUnsupported,
"Conversion from VariantType is not supported for Glue",
))
}
}

#[cfg(test)]
Expand Down
7 changes: 7 additions & 0 deletions crates/catalog/hms/src/schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,13 @@ impl SchemaVisitor for HiveSchemaBuilder {

Ok(hive_type)
}

fn variant(&mut self, _v: &iceberg::spec::VariantType) -> Result<Self::T> {
Err(Error::new(
ErrorKind::FeatureUnsupported,
"Conversion from VariantType is not supported for HMS",
))
}
}

#[cfg(test)]
Expand Down
6 changes: 5 additions & 1 deletion crates/iceberg/src/arrow/caching_delete_file_loader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ use crate::io::FileIO;
use crate::scan::{ArrowRecordBatchStream, FileScanTaskDeleteFile};
use crate::spec::{
DataContentType, Datum, ListType, MapType, NestedField, NestedFieldRef, PartnerAccessor,
PrimitiveType, Schema, SchemaRef, SchemaWithPartnerVisitor, StructType, Type,
PrimitiveType, Schema, SchemaRef, SchemaWithPartnerVisitor, StructType, Type, VariantType,
visit_schema_with_partner,
};
use crate::{Error, ErrorKind, Result};
Expand Down Expand Up @@ -527,6 +527,10 @@ impl SchemaWithPartnerVisitor<ArrayRef> for EqDelColumnProcessor<'_> {
fn primitive(&mut self, _primitive: &PrimitiveType, _partner: &ArrayRef) -> Result<()> {
Ok(())
}

fn variant(&mut self, _v: &VariantType, _partner: &ArrayRef) -> Result<()> {
Ok(())
}
}

struct EqDelRecordBatchPartnerAccessor;
Expand Down
6 changes: 5 additions & 1 deletion crates/iceberg/src/arrow/nan_val_cnt_visitor.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ use crate::Result;
use crate::arrow::{ArrowArrayAccessor, FieldMatchMode};
use crate::spec::{
ListType, MapType, NestedFieldRef, PrimitiveType, Schema, SchemaRef, SchemaWithPartnerVisitor,
StructType, visit_struct_with_partner,
StructType, VariantType, visit_struct_with_partner,
};

macro_rules! cast_and_update_cnt_map {
Expand Down Expand Up @@ -122,6 +122,10 @@ impl SchemaWithPartnerVisitor<ArrayRef> for NanValueCountVisitor {
Ok(())
}

fn variant(&mut self, _v: &VariantType, _col: &ArrayRef) -> Result<Self::T> {
Ok(())
}

fn after_struct_field(&mut self, field: &NestedFieldRef, partner: &ArrayRef) -> Result<()> {
let field_id = field.id;
count_float_nans!(partner, self, field_id);
Expand Down
157 changes: 151 additions & 6 deletions crates/iceberg/src/arrow/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -682,6 +682,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);
}
}
}

Expand Down Expand Up @@ -754,8 +758,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() {
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.
Expand All @@ -767,14 +809,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;
};
Expand All @@ -796,16 +846,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);
}
}

Expand Down Expand Up @@ -1835,7 +1885,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 {
Expand Down Expand Up @@ -2003,6 +2054,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'
Expand Down
27 changes: 26 additions & 1 deletion crates/iceberg/src/arrow/schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ use crate::error::Result;
use crate::spec::decimal_utils::i128_from_be_bytes;
use crate::spec::{
Datum, FIRST_FIELD_ID, ListType, MapType, NestedField, NestedFieldRef, PrimitiveLiteral,
PrimitiveType, Schema, SchemaVisitor, StructType, Type,
PrimitiveType, Schema, SchemaVisitor, StructType, Type, VariantType,
};
use crate::{Error, ErrorKind};

Expand Down Expand Up @@ -689,6 +689,16 @@ impl SchemaVisitor for ToArrowSchemaConverter {
}
}
}

fn variant(&mut self, _v: &VariantType) -> crate::Result<ArrowSchemaOrFieldOrType> {
// Variant is stored as a struct with two required binary fields (no field IDs on sub-fields).
// Uses Binary (not LargeBinary) matching the Parquet BINARY primitive directly.
let metadata_field = Field::new("metadata", DataType::Binary, false);
let value_field = Field::new("value", DataType::Binary, false);
Ok(ArrowSchemaOrFieldOrType::Type(DataType::Struct(
vec![metadata_field, value_field].into(),
)))
}
}

/// Convert iceberg schema to an arrow schema.
Expand Down Expand Up @@ -1697,6 +1707,15 @@ mod tests {
simple_field("map", map, false, "16"),
simple_field("struct", r#struct, false, "17"),
simple_field("uuid", DataType::FixedSizeBinary(16), false, "30"),
simple_field(
"v",
DataType::Struct(Fields::from(vec![
Field::new("metadata", DataType::Binary, false),
Field::new("value", DataType::Binary, false),
])),
true,
"31",
),
])
}

Expand Down Expand Up @@ -1880,6 +1899,12 @@ mod tests {
"name":"uuid",
"required":true,
"type":"uuid"
},
{
"id":31,
"name":"v",
"required":false,
"type":"variant"
}
],
"identifier-field-ids":[]
Expand Down
9 changes: 8 additions & 1 deletion crates/iceberg/src/arrow/value.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ use uuid::Uuid;
use super::get_field_id_from_metadata;
use crate::spec::{
ListType, Literal, Map, MapType, NestedField, PartnerAccessor, PrimitiveLiteral, PrimitiveType,
SchemaWithPartnerVisitor, Struct, StructType, Type, visit_struct_with_partner,
SchemaWithPartnerVisitor, Struct, StructType, Type, VariantType, visit_struct_with_partner,
visit_type_with_partner,
};
use crate::{Error, ErrorKind, Result};
Expand Down Expand Up @@ -426,6 +426,13 @@ impl SchemaWithPartnerVisitor<ArrayRef> for ArrowArrayToIcebergStructConverter {
}
}
}

fn variant(&mut self, _v: &VariantType, _partner: &ArrayRef) -> Result<Vec<Option<Literal>>> {
Err(Error::new(
ErrorKind::FeatureUnsupported,
"Converting variant Arrow array to Iceberg literal is not supported yet",
))
}
}

/// Defines how Arrow fields are matched with Iceberg fields when converting data.
Expand Down
Loading
Loading