diff --git a/crates/iceberg/src/table.rs b/crates/iceberg/src/table.rs index d4e696ce8..a7005f9b4 100644 --- a/crates/iceberg/src/table.rs +++ b/crates/iceberg/src/table.rs @@ -24,7 +24,7 @@ use crate::inspect::MetadataTable; use crate::io::FileIO; use crate::io::object_cache::ObjectCache; use crate::scan::TableScanBuilder; -use crate::spec::{TableMetadata, TableMetadataRef}; +use crate::spec::{SchemaRef, TableMetadata, TableMetadataRef}; use crate::{Error, ErrorKind, Result, TableIdent}; /// Builder to create table scan. @@ -235,6 +235,11 @@ impl Table { self.readonly } + /// Returns the current schema as a shared reference. + pub fn schema_ref(&self) -> SchemaRef { + self.metadata.current_schema().clone() + } + /// Create a reader for the table. pub fn reader_builder(&self) -> ArrowReaderBuilder { ArrowReaderBuilder::new(self.file_io.clone()) diff --git a/crates/integrations/datafusion/src/physical_plan/repartition.rs b/crates/integrations/datafusion/src/physical_plan/repartition.rs new file mode 100644 index 000000000..ade17a0a9 --- /dev/null +++ b/crates/integrations/datafusion/src/physical_plan/repartition.rs @@ -0,0 +1,791 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::HashSet; +use std::num::NonZeroUsize; +use std::sync::Arc; + +use datafusion::error::Result as DFResult; +use datafusion::physical_plan::expressions::Column; +use datafusion::physical_plan::repartition::RepartitionExec; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use iceberg::spec::{TableMetadata, TableMetadataRef, Transform}; + +/// Creates an Iceberg-aware repartition execution plan that optimizes data distribution +/// for parallel processing while respecting Iceberg table partitioning semantics. +/// +/// This function automatically determines the optimal partitioning strategy based on +/// the table's partition specification and sort order: +/// +/// ## Partitioning Strategies +/// +/// - **Unpartitioned tables**: Uses round-robin distribution to ensure balanced load +/// across all workers, maximizing parallelism for write operations. +/// +/// - **Hash partitioning**: Used for tables with identity transforms or bucket transforms: +/// - Identity partition columns (e.g., `PARTITIONED BY (user_id, category)`) +/// - Bucket columns from partition spec or sort order +/// - This ensures data co-location within partitions and buckets for optimal file clustering +/// +/// - **Round-robin partitioning**: Used for: +/// - Range-only partitions (e.g., date/time partitions that concentrate data) +/// - Tables with only temporal/range transforms that don't provide good distribution +/// - Unpartitioned or non-bucketed tables +/// +/// - **Mixed transforms**: Tables with both range and identity/bucket transforms use hash +/// partitioning on the identity/bucket columns for optimal distribution. +/// +/// ## Performance notes +/// +/// - Only repartitions when the input partitioning scheme differs from the desired strategy +/// - Only repartitions when the input partition count differs from the target +/// - Requires explicit target partition count for deterministic behavior +/// - Preserves column order (partitions first, then buckets) for consistent file layout +/// +/// # Arguments +/// +/// * `input` - The input execution plan providing data to be repartitioned (should already be projected to match table schema) +/// * `table_metadata` - The Iceberg table metadata containing partition spec and sort order +/// * `target_partitions` - Target number of partitions for parallel processing (must be > 0) +/// +/// # Returns +/// +/// An execution plan that will apply the optimal partitioning strategy during execution, +/// or the original input plan unchanged if no repartitioning is needed. +/// +/// # Example +/// +/// ```ignore +/// let repartitioned_plan = repartition( +/// input_plan, +/// table.metadata_ref(), +/// 4, // Explicit partition count +/// )?; +/// ``` +pub(crate) fn repartition( + input: Arc, + table_metadata: TableMetadataRef, + target_partitions: NonZeroUsize, +) -> DFResult> { + let partitioning_strategy = + determine_partitioning_strategy(&input, &table_metadata, target_partitions)?; + + if !needs_repartitioning(&input, &partitioning_strategy) { + return Ok(input); + } + + Ok(Arc::new(RepartitionExec::try_new( + input, + partitioning_strategy, + )?)) +} + +/// Returns whether repartitioning is actually needed by comparing input and desired partitioning +fn needs_repartitioning(input: &Arc, desired: &Partitioning) -> bool { + let input_partitioning = input.properties().output_partitioning(); + match (input_partitioning, desired) { + (Partitioning::RoundRobinBatch(a), Partitioning::RoundRobinBatch(b)) => a != b, + (Partitioning::Hash(a_exprs, a_n), Partitioning::Hash(b_exprs, b_n)) => { + a_n != b_n || !same_columns(a_exprs, b_exprs) + } + _ => true, + } +} + +/// Helper function to check if two sets of column expressions are the same +fn same_columns( + a_exprs: &[Arc], + b_exprs: &[Arc], +) -> bool { + if a_exprs.len() != b_exprs.len() { + return false; + } + a_exprs + .iter() + .zip(b_exprs.iter()) + .all(|(a, b)| a.as_any().downcast_ref::() == b.as_any().downcast_ref::()) +} + +/// Determines the optimal partitioning strategy based on table metadata. +/// +/// This function analyzes the table's partition specification and sort order to select +/// the most appropriate DataFusion partitioning strategy for insert operations. +/// +/// ## Partitioning Strategy Logic +/// +/// The strategy is determined by analyzing the table's partition transforms: +/// +/// - **Hash partitioning**: Used only when there are identity transforms (direct column partitioning) +/// or bucket transforms that provide good data distribution: +/// 1. Identity partition columns (e.g., `PARTITIONED BY (user_id, category)`) +/// 2. Bucket columns from partition spec (e.g., `bucket(16, user_id)`) +/// 3. Bucket columns from sort order +/// +/// This ensures data co-location within partitions and buckets for optimal file clustering. +/// +/// - **Round-robin partitioning**: Used for: +/// - Unpartitioned tables +/// - Range-only partitions (e.g., date/time partitions that concentrate data) +/// - Tables with only temporal/range transforms that don't provide good distribution +/// - Tables with no suitable hash columns +/// +/// ## Column Priority and Deduplication +/// +/// When multiple column sources are available, they are combined in this order: +/// 1. Partition identity columns (highest priority) +/// 2. Bucket columns from partition spec +/// 3. Bucket columns from sort order +/// +/// Duplicate columns are automatically removed while preserving the priority order. +/// +/// ## Fallback Behavior +/// +/// If no suitable hash columns are found (e.g., unpartitioned, range-only, or non-bucketed table), +/// falls back to round-robin batch partitioning for even load distribution. +fn determine_partitioning_strategy( + input: &Arc, + table_metadata: &TableMetadata, + target_partitions: NonZeroUsize, +) -> DFResult { + let partition_spec = table_metadata.default_partition_spec(); + let table_schema = table_metadata.current_schema(); + + let names_iter: Box> = { + // Partition identity columns + let part_names = partition_spec.fields().iter().filter_map(|pf| { + if matches!(pf.transform, Transform::Identity) { + table_schema + .field_by_id(pf.source_id) + .map(|sf| sf.name.as_str()) + } else { + None + } + }); + // Bucket columns from partition spec + let bucket_names_part = partition_spec.fields().iter().filter_map(|pf| { + if let Transform::Bucket(_) = pf.transform { + table_schema + .field_by_id(pf.source_id) + .map(|sf| sf.name.as_str()) + } else { + None + } + }); + Box::new(part_names.chain(bucket_names_part)) + }; + + // Order: partitions first, then buckets + // Deduplicate while preserving order + let input_schema = input.schema(); + let mut seen = HashSet::new(); + let hash_exprs: Vec> = names_iter + .filter(|name| seen.insert(*name)) + .map(|name| { + let idx = input_schema + .index_of(name) + .map_err(|e| { + datafusion::error::DataFusionError::Plan(format!( + "Column '{}' not found in input schema. Ensure projection happens before repartitioning. Error: {}", + name, e + )) + })?; + Ok(Arc::new(Column::new(name, idx)) + as Arc) + }) + .collect::>()?; + + if !hash_exprs.is_empty() { + return Ok(Partitioning::Hash(hash_exprs, target_partitions.get())); + } + + // Fallback to round-robin for unpartitioned, non-bucketed tables, and range-only partitions + Ok(Partitioning::RoundRobinBatch(target_partitions.get())) +} + +#[cfg(test)] +mod tests { + use datafusion::arrow::datatypes::{ + DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, + }; + use datafusion::execution::TaskContext; + use datafusion::physical_plan::empty::EmptyExec; + use iceberg::TableIdent; + use iceberg::io::FileIO; + use iceberg::spec::{ + NestedField, NullOrder, PrimitiveType, Schema, SortDirection, SortField, SortOrder, + Transform, Type, + }; + use iceberg::table::Table; + + use super::*; + + fn create_test_table() -> Table { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 2, + "data", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .build() + .unwrap(); + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + + Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/metadata.json".to_string()) + .build() + .unwrap() + } + + fn create_test_arrow_schema() -> Arc { + Arc::new(ArrowSchema::new(vec![ + ArrowField::new("id", ArrowDataType::Int64, false), + ArrowField::new("data", ArrowDataType::Utf8, false), + ])) + } + + #[tokio::test] + async fn test_repartition_unpartitioned_table() { + let table = create_test_table(); + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let repartitioned_plan = repartition( + input.clone(), + table.metadata().current_schema().clone(), + table.metadata_ref(), + 4, + ) + .unwrap(); + + assert_ne!(input.name(), repartitioned_plan.name()); + assert_eq!(repartitioned_plan.name(), "RepartitionExec"); + } + + #[tokio::test] + async fn test_repartition_explicit_partitions() { + let table = create_test_table(); + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let repartitioned_plan = repartition( + input, + table.metadata().current_schema().clone(), + table.metadata_ref(), + 8, + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::RoundRobinBatch(n) => { + assert_eq!(*n, 8); + } + _ => panic!("Expected RoundRobinBatch partitioning"), + } + } + + #[tokio::test] + async fn test_repartition_zero_partitions_fails() { + let table = create_test_table(); + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let result = repartition( + input, + table.metadata().current_schema().clone(), + table.metadata_ref(), + 0, + ); + + assert!(result.is_err()); + assert!( + result + .unwrap_err() + .to_string() + .contains("requires target_partitions > 0") + ); + } + + #[tokio::test] + async fn test_partition_count_validation() { + let table = create_test_table(); + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let target_partitions = 16; + let repartitioned_plan = repartition( + input, + table.metadata().current_schema().clone(), + table.metadata_ref(), + target_partitions, + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::RoundRobinBatch(n) => { + assert_eq!(*n, target_partitions); + } + _ => panic!("Expected RoundRobinBatch partitioning"), + } + } + + #[tokio::test] + async fn test_datafusion_repartitioning_integration() { + let table = create_test_table(); + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + + let repartitioned_plan = repartition( + input, + table.metadata().current_schema().clone(), + table.metadata_ref(), + 3, + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::RoundRobinBatch(n) => { + assert_eq!(*n, 3, "Should use round-robin for unpartitioned table"); + } + _ => panic!("Expected RoundRobinBatch partitioning for unpartitioned table"), + } + + let task_ctx = Arc::new(TaskContext::default()); + let stream = repartitioned_plan.execute(0, task_ctx.clone()).unwrap(); + + // Verify the stream was created successfully + assert!(!stream.schema().fields().is_empty()); + } + + #[tokio::test] + async fn test_bucket_aware_partitioning() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 2, + "category", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(); + + let sort_order = SortOrder::builder() + .with_order_id(1) + .with_sort_field(SortField { + source_id: 2, + transform: Transform::Bucket(4), + direction: SortDirection::Ascending, + null_order: NullOrder::First, + }) + .build(&schema) + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .build() + .unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/bucketed_table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "bucketed_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/bucketed_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("id", ArrowDataType::Int64, false), + ArrowField::new("category", ArrowDataType::Utf8, false), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + let repartitioned_plan = repartition( + input, + table.metadata().current_schema().clone(), + table.metadata_ref(), + 4, + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + assert!( + matches!(partitioning, Partitioning::Hash(_, _)), + "Should use hash partitioning for bucketed table" + ); + } + + #[tokio::test] + async fn test_combined_partition_and_bucket_strategy() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::required( + 2, + "user_id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 3, + "amount", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .add_partition_field("date", "date", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let sort_order = SortOrder::builder() + .with_order_id(1) + .with_sort_field(SortField { + source_id: 2, + transform: Transform::Bucket(8), + direction: SortDirection::Ascending, + null_order: NullOrder::First, + }) + .build(&schema) + .unwrap(); + + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/partitioned_bucketed_table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "partitioned_bucketed_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/partitioned_bucketed_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("date", ArrowDataType::Date32, false), + ArrowField::new("user_id", ArrowDataType::Int64, false), + ArrowField::new("amount", ArrowDataType::Int64, false), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + let repartitioned_plan = repartition( + input, + table.metadata().current_schema().clone(), + table.metadata_ref(), + 4, + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::Hash(exprs, _) => { + assert_eq!( + exprs.len(), + 2, + "Should have both partition and bucket columns" + ); + + let column_names: Vec = exprs + .iter() + .filter_map(|expr| { + expr.as_any() + .downcast_ref::() + .map(|col| col.name().to_string()) + }) + .collect(); + + assert!( + column_names.contains(&"date".to_string()), + "Should include partition column 'date'" + ); + assert!( + column_names.contains(&"user_id".to_string()), + "Should include bucket column 'user_id'" + ); + } + _ => panic!("Expected Hash partitioning for partitioned+bucketed table"), + } + } + + #[tokio::test] + async fn test_none_distribution_mode_fallback() { + let schema = Schema::builder() + .with_fields(vec![Arc::new(NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + ))]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .build() + .unwrap(); + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + + let mut properties = std::collections::HashMap::new(); + properties.insert("write.distribution-mode".to_string(), "none".to_string()); + + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/none_table".to_string(), + iceberg::spec::FormatVersion::V2, + properties, + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "none_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/none_metadata.json".to_string()) + .build() + .unwrap(); + + let input = Arc::new(EmptyExec::new(create_test_arrow_schema())); + let repartitioned_plan = repartition( + input, + table.metadata().current_schema().clone(), + table.metadata_ref(), + 4, + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + assert!( + matches!(partitioning, Partitioning::RoundRobinBatch(_)), + "Should use round-robin for 'none' distribution mode" + ); + } + + #[tokio::test] + async fn test_schema_ref_convenience_method() { + let table = create_test_table(); + + let schema_ref_1 = table.schema_ref(); + let schema_ref_2 = Arc::clone(table.metadata().current_schema()); + + assert!( + Arc::ptr_eq(&schema_ref_1, &schema_ref_2), + "schema_ref() should return the same Arc as manual approach" + ); + } + + #[tokio::test] + async fn test_range_only_partitions_use_round_robin() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::required( + 2, + "amount", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .add_partition_field("date", "date_day", Transform::Day) + .unwrap() + .build() + .unwrap(); + + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/range_only_table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "range_only_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/range_only_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("date", ArrowDataType::Date32, false), + ArrowField::new("amount", ArrowDataType::Int64, false), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + let repartitioned_plan = repartition( + input, + table.metadata().current_schema().clone(), + table.metadata_ref(), + 4, + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + assert!( + matches!(partitioning, Partitioning::RoundRobinBatch(_)), + "Should use round-robin for range-only partitions" + ); + } + + #[tokio::test] + async fn test_mixed_transforms_use_hash_partitioning() { + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::required( + 2, + "user_id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::required( + 3, + "amount", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(schema.clone()) + .add_partition_field("date", "date_day", Transform::Day) + .unwrap() + .add_partition_field("user_id", "user_id", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let sort_order = iceberg::spec::SortOrder::builder().build(&schema).unwrap(); + let table_metadata_builder = iceberg::spec::TableMetadataBuilder::new( + schema, + partition_spec, + sort_order, + "/test/mixed_transforms_table".to_string(), + iceberg::spec::FormatVersion::V2, + std::collections::HashMap::new(), + ) + .unwrap(); + + let table_metadata = table_metadata_builder.build().unwrap(); + let table = Table::builder() + .metadata(table_metadata.metadata) + .identifier(TableIdent::from_strs(["test", "mixed_transforms_table"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .metadata_location("/test/mixed_transforms_metadata.json".to_string()) + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("date", ArrowDataType::Date32, false), + ArrowField::new("user_id", ArrowDataType::Int64, false), + ArrowField::new("amount", ArrowDataType::Int64, false), + ])); + let input = Arc::new(EmptyExec::new(arrow_schema)); + let repartitioned_plan = repartition( + input, + table.metadata().current_schema().clone(), + table.metadata_ref(), + 4, + ) + .unwrap(); + + let partitioning = repartitioned_plan.properties().output_partitioning(); + match partitioning { + Partitioning::Hash(exprs, _) => { + assert_eq!( + exprs.len(), + 1, + "Should have one hash column (user_id identity transform)" + ); + let column_names: Vec = exprs + .iter() + .filter_map(|expr| { + expr.as_any() + .downcast_ref::() + .map(|col| col.name().to_string()) + }) + .collect(); + assert!( + column_names.contains(&"user_id".to_string()), + "Should include identity transform column 'user_id'" + ); + } + _ => panic!("Expected Hash partitioning for table with identity transforms"), + } + } +}