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
2 changes: 1 addition & 1 deletion datafusion/core/src/physical_planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1245,7 +1245,7 @@ impl DefaultPhysicalPlanner {
}

// N Children
LogicalPlan::Union(_) => Arc::new(UnionExec::new(children.vec())),
LogicalPlan::Union(_) => Arc::new(UnionExec::new(children.vec())?),
LogicalPlan::Extension(Extension { node }) => {
let mut maybe_plan = None;
let children = children.vec();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1783,7 +1783,7 @@ fn union_to_interleave() -> Result<()> {
);

// Union
let plan = Arc::new(UnionExec::new(vec![left, right]));
let plan = Arc::new(UnionExec::new(vec![left, right])?);

// final agg
let plan =
Expand Down Expand Up @@ -1827,7 +1827,7 @@ fn union_not_to_interleave() -> Result<()> {
);

// Union
let plan = Arc::new(UnionExec::new(vec![left, right]));
let plan = Arc::new(UnionExec::new(vec![left, right])?);

// final agg
let plan =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -357,7 +357,7 @@ mod test {
async fn test_statistic_by_partition_of_union() -> Result<()> {
let scan = create_scan_exec_with_statistics(None, Some(2)).await;
let union_exec: Arc<dyn ExecutionPlan> =
Arc::new(UnionExec::new(vec![scan.clone(), scan]));
Arc::new(UnionExec::new(vec![scan.clone(), scan])?);
let statistics = (0..union_exec.output_partitioning().partition_count())
.map(|idx| union_exec.partition_statistics(Some(idx)))
.collect::<Result<Vec<_>>>()?;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1535,7 +1535,7 @@ fn test_sort_preserving_after_projection() -> Result<()> {
#[test]
fn test_union_after_projection() -> Result<()> {
let csv = create_simple_csv_exec();
let union = Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv]));
let union = Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])?);
let projection: Arc<dyn ExecutionPlan> = Arc::new(ProjectionExec::try_new(
vec![
ProjectionExpr::new(Arc::new(Column::new("c", 2)), "c".to_string()),
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/tests/physical_optimizer/test_utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -304,7 +304,7 @@ pub fn sort_preserving_merge_exec_with_fetch(
}

pub fn union_exec(input: Vec<Arc<dyn ExecutionPlan>>) -> Arc<dyn ExecutionPlan> {
Arc::new(UnionExec::new(input))
Arc::new(UnionExec::new(input).expect("Failed to create UnionExec"))
}

pub fn local_limit_exec(
Expand Down
4 changes: 2 additions & 2 deletions datafusion/physical-plan/src/repartition/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1783,7 +1783,7 @@ mod test {
let source1 = sorted_memory_exec(&schema, sort_exprs.clone());
let source2 = sorted_memory_exec(&schema, sort_exprs);
// output has multiple partitions, and is sorted
let union = UnionExec::new(vec![source1, source2]);
let union = UnionExec::new(vec![source1, source2])?;
let exec =
RepartitionExec::try_new(Arc::new(union), Partitioning::RoundRobinBatch(10))
.unwrap()
Expand Down Expand Up @@ -1825,7 +1825,7 @@ mod test {
let source1 = memory_exec(&schema);
let source2 = memory_exec(&schema);
// output has multiple partitions, but is not sorted
let union = UnionExec::new(vec![source1, source2]);
let union = UnionExec::new(vec![source1, source2])?;
let exec =
RepartitionExec::try_new(Arc::new(union), Partitioning::RoundRobinBatch(10))
.unwrap()
Expand Down
90 changes: 75 additions & 15 deletions datafusion/physical-plan/src/union.rs
Original file line number Diff line number Diff line change
Expand Up @@ -101,19 +101,23 @@ pub struct UnionExec {

impl UnionExec {
/// Create a new UnionExec
pub fn new(inputs: Vec<Arc<dyn ExecutionPlan>>) -> Self {
let schema = union_schema(&inputs);
pub fn new(inputs: Vec<Arc<dyn ExecutionPlan>>) -> Result<Self> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is technically an API change -- maybe to make it easier on others, we can make a new function called try_new that has the error checking, and deprecate the existing new function per https://datafusion.apache.org/contributor-guide/api-health.html#deprecation-guidelines

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point on the API lifecycle. On separate note, can we make the new try_new method return Box<<dyn ExecutionPlan>>? This would allow it to return the only child in case input vector is a singleton. There is no point keeping UnionExec(a) in the plan.
Or maybe, the new method can simply require the input to have at least two elements?

if inputs.is_empty() {
return exec_err!("UnionExec requires at least one input");
}

let schema = union_schema(&inputs)?;
// The schema of the inputs and the union schema is consistent when:
// - They have the same number of fields, and
// - Their fields have same types at the same indices.
// Here, we know that schemas are consistent and the call below can
// not return an error.
let cache = Self::compute_properties(&inputs, schema).unwrap();
UnionExec {
Ok(UnionExec {
inputs,
metrics: ExecutionPlanMetricsSet::new(),
cache,
}
})
}

/// Get inputs of the execution plan
Expand Down Expand Up @@ -220,7 +224,7 @@ impl ExecutionPlan for UnionExec {
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(UnionExec::new(children)))
Ok(Arc::new(UnionExec::new(children)?))
}

fn execute(
Expand Down Expand Up @@ -319,7 +323,7 @@ impl ExecutionPlan for UnionExec {
.map(|child| make_with_child(projection, child))
.collect::<Result<Vec<_>>>()?;

Ok(Some(Arc::new(UnionExec::new(new_children))))
Ok(Some(Arc::new(UnionExec::new(new_children)?)))
}
}

Expand Down Expand Up @@ -373,7 +377,7 @@ impl InterleaveExec {
"Not all InterleaveExec children have a consistent hash partitioning"
);
}
let cache = Self::compute_properties(&inputs);
let cache = Self::compute_properties(&inputs)?;
Ok(InterleaveExec {
inputs,
metrics: ExecutionPlanMetricsSet::new(),
Expand All @@ -387,17 +391,17 @@ impl InterleaveExec {
}

/// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
fn compute_properties(inputs: &[Arc<dyn ExecutionPlan>]) -> PlanProperties {
let schema = union_schema(inputs);
fn compute_properties(inputs: &[Arc<dyn ExecutionPlan>]) -> Result<PlanProperties> {
let schema = union_schema(inputs)?;
let eq_properties = EquivalenceProperties::new(schema);
// Get output partitioning:
let output_partitioning = inputs[0].output_partitioning().clone();
PlanProperties::new(
Ok(PlanProperties::new(
eq_properties,
output_partitioning,
emission_type_from_children(inputs),
boundedness_from_children(inputs),
)
))
}
}

Expand Down Expand Up @@ -538,7 +542,11 @@ pub fn can_interleave<T: Borrow<Arc<dyn ExecutionPlan>>>(
.all(|partition| partition == *reference)
}

fn union_schema(inputs: &[Arc<dyn ExecutionPlan>]) -> SchemaRef {
fn union_schema(inputs: &[Arc<dyn ExecutionPlan>]) -> Result<SchemaRef> {
if inputs.is_empty() {
return exec_err!("Cannot create union schema from empty inputs");
}

let first_schema = inputs[0].schema();

let fields = (0..first_schema.fields().len())
Expand Down Expand Up @@ -581,7 +589,7 @@ fn union_schema(inputs: &[Arc<dyn ExecutionPlan>]) -> SchemaRef {
.flat_map(|i| i.schema().metadata().clone().into_iter())
.collect();

Arc::new(Schema::new_with_metadata(fields, all_metadata_merged))
Ok(Arc::new(Schema::new_with_metadata(fields, all_metadata_merged)))
}

/// CombinedRecordBatchStream can be used to combine a Vec of SendableRecordBatchStreams into one
Expand Down Expand Up @@ -710,7 +718,7 @@ mod tests {
let csv = test::scan_partitioned(4);
let csv2 = test::scan_partitioned(5);

let union_exec = Arc::new(UnionExec::new(vec![csv, csv2]));
let union_exec = Arc::new(UnionExec::new(vec![csv, csv2])?);

// Should have 9 partitions and 9 output batches
assert_eq!(
Expand Down Expand Up @@ -892,7 +900,7 @@ mod tests {
let mut union_expected_eq = EquivalenceProperties::new(Arc::clone(&schema));
union_expected_eq.add_orderings(union_expected_orderings);

let union = UnionExec::new(vec![child1, child2]);
let union = UnionExec::new(vec![child1, child2])?;
let union_eq_properties = union.properties().equivalence_properties();
let err_msg = format!(
"Error in test id: {:?}, test case: {:?}",
Expand All @@ -916,4 +924,56 @@ mod tests {
assert!(lhs_orderings.contains(rhs_ordering), "{}", err_msg);
}
}

#[test]
fn test_union_empty_inputs() {
// Test that UnionExec::new fails with empty inputs
let result = UnionExec::new(vec![]);
assert!(result.is_err());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the assertion check for is_err is redundant as unwrap_err will panic if result is not an err

assert!(result
.unwrap_err()
.to_string()
.contains("UnionExec requires at least one input"));
}

#[test]
fn test_union_schema_empty_inputs() {
// Test that union_schema fails with empty inputs
let result = union_schema(&[]);
assert!(result.is_err());
assert!(result
.unwrap_err()
.to_string()
.contains("Cannot create union schema from empty inputs"));
}

#[test]
fn test_union_single_input() -> Result<()> {
// Test that UnionExec works with a single input
let schema = create_test_schema()?;
let memory_exec = Arc::new(TestMemoryExec::try_new(&[], schema.clone(), None)?);
let union = UnionExec::new(vec![memory_exec])?;

// Check that schema is correct
assert_eq!(union.schema(), schema);

Ok(())
}

#[test]
fn test_union_multiple_inputs_still_works() -> Result<()> {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
fn test_union_multiple_inputs_still_works() -> Result<()> {
fn test_union_schema_multiple_inputs() -> Result<()> {

// Test that existing functionality with multiple inputs still works
let schema = create_test_schema()?;
let memory_exec1 = Arc::new(TestMemoryExec::try_new(&[], schema.clone(), None)?);
let memory_exec2 = Arc::new(TestMemoryExec::try_new(&[], schema.clone(), None)?);

let union = UnionExec::new(vec![memory_exec1, memory_exec2])?;

// Check that schema is correct
assert_eq!(union.schema(), schema);
// Check that we have 2 inputs
assert_eq!(union.inputs().len(), 2);

Ok(())
}
}
Loading