1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713
// 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.
// Some of these functions reference the Postgres documentation
// or implementation to ensure compatibility and are subject to
// the Postgres license.
//! The Union operator combines multiple inputs with the same schema
use std::pin::Pin;
use std::task::{Context, Poll};
use std::{any::Any, sync::Arc};
use arrow::{
datatypes::{Field, Schema, SchemaRef},
record_batch::RecordBatch,
};
use datafusion_common::{DFSchemaRef, DataFusionError};
use futures::Stream;
use itertools::Itertools;
use log::{debug, trace, warn};
use super::{
expressions::PhysicalSortExpr,
metrics::{ExecutionPlanMetricsSet, MetricsSet},
ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream,
SendableRecordBatchStream, Statistics,
};
use crate::physical_plan::common::get_meet_of_orderings;
use crate::physical_plan::stream::ObservedStream;
use crate::physical_plan::{expressions, metrics::BaselineMetrics};
use datafusion_common::Result;
use datafusion_execution::TaskContext;
use tokio::macros::support::thread_rng_n;
/// `UnionExec`: `UNION ALL` execution plan.
///
/// `UnionExec` combines multiple inputs with the same schema by
/// concatenating the partitions. It does not mix or copy data within
/// or across partitions. Thus if the input partitions are sorted, the
/// output partitions of the union are also sorted.
///
/// For example, given a `UnionExec` of two inputs, with `N`
/// partitions, and `M` partitions, there will be `N+M` output
/// partitions. The first `N` output partitions are from Input 1
/// partitions, and then next `M` output partitions are from Input 2.
///
/// ```text
/// ▲ ▲ ▲ ▲
/// │ │ │ │
/// Output │ ... │ │ │
/// Partitions │0 │N-1 │ N │N+M-1
///(passes through ┌────┴───────┴───────────┴─────────┴───┐
/// the N+M input │ UnionExec │
/// partitions) │ │
/// └──────────────────────────────────────┘
/// ▲
/// │
/// │
/// Input ┌────────┬─────┴────┬──────────┐
/// Partitions │ ... │ │ ... │
/// 0 │ │ N-1 │ 0 │ M-1
/// ┌────┴────────┴───┐ ┌───┴──────────┴───┐
/// │ │ │ │
/// │ │ │ │
/// │ │ │ │
/// │ │ │ │
/// │ │ │ │
/// │ │ │ │
/// │Input 1 │ │Input 2 │
/// └─────────────────┘ └──────────────────┘
/// ```
#[derive(Debug)]
pub struct UnionExec {
/// Input execution plan
inputs: Vec<Arc<dyn ExecutionPlan>>,
/// Execution metrics
metrics: ExecutionPlanMetricsSet,
/// Schema of Union
schema: SchemaRef,
}
impl UnionExec {
/// Create a new UnionExec with specified schema.
/// The `schema` should always be a subset of the schema of `inputs`,
/// otherwise, an error will be returned.
pub fn try_new_with_schema(
inputs: Vec<Arc<dyn ExecutionPlan>>,
schema: DFSchemaRef,
) -> Result<Self> {
let mut exec = Self::new(inputs);
let exec_schema = exec.schema();
let fields = schema
.fields()
.iter()
.map(|dff| {
exec_schema
.field_with_name(dff.name())
.cloned()
.map_err(|_| {
DataFusionError::Internal(format!(
"Cannot find the field {:?} in child schema",
dff.name()
))
})
})
.collect::<Result<Vec<Field>>>()?;
let schema = Arc::new(Schema::new_with_metadata(
fields,
exec.schema().metadata().clone(),
));
exec.schema = schema;
Ok(exec)
}
/// Create a new UnionExec
pub fn new(inputs: Vec<Arc<dyn ExecutionPlan>>) -> Self {
let schema = union_schema(&inputs);
UnionExec {
inputs,
metrics: ExecutionPlanMetricsSet::new(),
schema,
}
}
/// Get inputs of the execution plan
pub fn inputs(&self) -> &Vec<Arc<dyn ExecutionPlan>> {
&self.inputs
}
}
impl ExecutionPlan for UnionExec {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
/// Specifies whether this plan generates an infinite stream of records.
/// If the plan does not support pipelining, but its input(s) are
/// infinite, returns an error to indicate this.
fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
Ok(children.iter().any(|x| *x))
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
self.inputs.clone()
}
/// Output of the union is the combination of all output partitions of the inputs
fn output_partitioning(&self) -> Partitioning {
// Output the combination of all output partitions of the inputs if the Union is not partition aware
let num_partitions = self
.inputs
.iter()
.map(|plan| plan.output_partitioning().partition_count())
.sum();
Partitioning::UnknownPartitioning(num_partitions)
}
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
// The output ordering is the "meet" of its input orderings.
// The meet is the finest ordering that satisfied by all the input
// orderings, see https://en.wikipedia.org/wiki/Join_and_meet.
get_meet_of_orderings(&self.inputs)
}
fn maintains_input_order(&self) -> Vec<bool> {
// If the Union has an output ordering, it maintains at least one
// child's ordering (i.e. the meet).
// For instance, assume that the first child is SortExpr('a','b','c'),
// the second child is SortExpr('a','b') and the third child is
// SortExpr('a','b'). The output ordering would be SortExpr('a','b'),
// which is the "meet" of all input orderings. In this example, this
// function will return vec![false, true, true], indicating that we
// preserve the orderings for the 2nd and the 3rd children.
if let Some(output_ordering) = self.output_ordering() {
self.inputs()
.iter()
.map(|child| {
if let Some(child_ordering) = child.output_ordering() {
output_ordering.len() == child_ordering.len()
} else {
false
}
})
.collect()
} else {
vec![false; self.inputs().len()]
}
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(UnionExec::new(children)))
}
fn execute(
&self,
mut partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
trace!("Start UnionExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id());
let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
// record the tiny amount of work done in this function so
// elapsed_compute is reported as non zero
let elapsed_compute = baseline_metrics.elapsed_compute().clone();
let _timer = elapsed_compute.timer(); // record on drop
// find partition to execute
for input in self.inputs.iter() {
// Calculate whether partition belongs to the current partition
if partition < input.output_partitioning().partition_count() {
let stream = input.execute(partition, context)?;
debug!("Found a Union partition to execute");
return Ok(Box::pin(ObservedStream::new(stream, baseline_metrics)));
} else {
partition -= input.output_partitioning().partition_count();
}
}
warn!("Error in Union: Partition {} not found", partition);
Err(DataFusionError::Execution(format!(
"Partition {partition} not found in Union"
)))
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(f, "UnionExec")
}
}
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
fn statistics(&self) -> Statistics {
self.inputs
.iter()
.map(|ep| ep.statistics())
.reduce(stats_union)
.unwrap_or_default()
}
fn benefits_from_input_partitioning(&self) -> bool {
false
}
}
/// Combines multiple input streams by interleaving them.
///
/// This only works if all inputs have the same hash-partitioning.
///
/// # Data Flow
/// ```text
/// +---------+
/// | |---+
/// | Input 1 | |
/// | |-------------+
/// +---------+ | |
/// | | +---------+
/// +------------------>| |
/// +---------------->| Combine |-->
/// | +-------------->| |
/// | | | +---------+
/// +---------+ | | |
/// | |-----+ | |
/// | Input 2 | | |
/// | |---------------+
/// +---------+ | | |
/// | | | +---------+
/// | +-------->| |
/// | +------>| Combine |-->
/// | +---->| |
/// | | +---------+
/// +---------+ | |
/// | |-------+ |
/// | Input 3 | |
/// | |-----------------+
/// +---------+
/// ```
#[derive(Debug)]
pub struct InterleaveExec {
/// Input execution plan
inputs: Vec<Arc<dyn ExecutionPlan>>,
/// Execution metrics
metrics: ExecutionPlanMetricsSet,
/// Schema of Interleave
schema: SchemaRef,
}
impl InterleaveExec {
/// Create a new InterleaveExec
pub fn try_new(inputs: Vec<Arc<dyn ExecutionPlan>>) -> Result<Self> {
let schema = union_schema(&inputs);
if !can_interleave(&inputs) {
return Err(DataFusionError::Internal(String::from(
"Not all InterleaveExec children have a consistent hash partitioning",
)));
}
Ok(InterleaveExec {
inputs,
metrics: ExecutionPlanMetricsSet::new(),
schema,
})
}
/// Get inputs of the execution plan
pub fn inputs(&self) -> &Vec<Arc<dyn ExecutionPlan>> {
&self.inputs
}
}
impl ExecutionPlan for InterleaveExec {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
/// Specifies whether this plan generates an infinite stream of records.
/// If the plan does not support pipelining, but its input(s) are
/// infinite, returns an error to indicate this.
fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
Ok(children.iter().any(|x| *x))
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
self.inputs.clone()
}
/// All inputs must have the same partitioning. The output partioning of InterleaveExec is the same as the inputs
/// (NOT combined). E.g. if there are 10 inputs where each is `Hash(3)`-partitioned, InterleaveExec is also
/// `Hash(3)`-partitioned.
fn output_partitioning(&self) -> Partitioning {
self.inputs[0].output_partitioning()
}
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
None
}
fn maintains_input_order(&self) -> Vec<bool> {
vec![false; self.inputs().len()]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(InterleaveExec::try_new(children)?))
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
trace!("Start InterleaveExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id());
let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
// record the tiny amount of work done in this function so
// elapsed_compute is reported as non zero
let elapsed_compute = baseline_metrics.elapsed_compute().clone();
let _timer = elapsed_compute.timer(); // record on drop
let mut input_stream_vec = vec![];
for input in self.inputs.iter() {
if partition < input.output_partitioning().partition_count() {
input_stream_vec.push(input.execute(partition, context.clone())?);
} else {
// Do not find a partition to execute
break;
}
}
if input_stream_vec.len() == self.inputs.len() {
let stream = Box::pin(CombinedRecordBatchStream::new(
self.schema(),
input_stream_vec,
));
return Ok(Box::pin(ObservedStream::new(stream, baseline_metrics)));
}
warn!("Error in InterleaveExec: Partition {} not found", partition);
Err(DataFusionError::Execution(format!(
"Partition {partition} not found in InterleaveExec"
)))
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(f, "InterleaveExec")
}
}
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
fn statistics(&self) -> Statistics {
self.inputs
.iter()
.map(|ep| ep.statistics())
.reduce(stats_union)
.unwrap_or_default()
}
fn benefits_from_input_partitioning(&self) -> bool {
false
}
}
/// If all the input partitions have the same Hash partition spec with the first_input_partition
/// The InterleaveExec is partition aware.
///
/// It might be too strict here in the case that the input partition specs are compatible but not exactly the same.
/// For example one input partition has the partition spec Hash('a','b','c') and
/// other has the partition spec Hash('a'), It is safe to derive the out partition with the spec Hash('a','b','c').
pub fn can_interleave(inputs: &[Arc<dyn ExecutionPlan>]) -> bool {
if inputs.is_empty() {
return false;
}
let first_input_partition = inputs[0].output_partitioning();
matches!(first_input_partition, Partitioning::Hash(_, _))
&& inputs
.iter()
.map(|plan| plan.output_partitioning())
.all(|partition| partition == first_input_partition)
}
fn union_schema(inputs: &[Arc<dyn ExecutionPlan>]) -> SchemaRef {
let fields: Vec<Field> = (0..inputs[0].schema().fields().len())
.map(|i| {
inputs
.iter()
.filter_map(|input| {
if input.schema().fields().len() > i {
Some(input.schema().field(i).clone())
} else {
None
}
})
.find_or_first(|f| f.is_nullable())
.unwrap()
})
.collect();
Arc::new(Schema::new_with_metadata(
fields,
inputs[0].schema().metadata().clone(),
))
}
/// CombinedRecordBatchStream can be used to combine a Vec of SendableRecordBatchStreams into one
struct CombinedRecordBatchStream {
/// Schema wrapped by Arc
schema: SchemaRef,
/// Stream entries
entries: Vec<SendableRecordBatchStream>,
}
impl CombinedRecordBatchStream {
/// Create an CombinedRecordBatchStream
pub fn new(schema: SchemaRef, entries: Vec<SendableRecordBatchStream>) -> Self {
Self { schema, entries }
}
}
impl RecordBatchStream for CombinedRecordBatchStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
impl Stream for CombinedRecordBatchStream {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
use Poll::*;
let start = thread_rng_n(self.entries.len() as u32) as usize;
let mut idx = start;
for _ in 0..self.entries.len() {
let stream = self.entries.get_mut(idx).unwrap();
match Pin::new(stream).poll_next(cx) {
Ready(Some(val)) => return Ready(Some(val)),
Ready(None) => {
// Remove the entry
self.entries.swap_remove(idx);
// Check if this was the last entry, if so the cursor needs
// to wrap
if idx == self.entries.len() {
idx = 0;
} else if idx < start && start <= self.entries.len() {
// The stream being swapped into the current index has
// already been polled, so skip it.
idx = idx.wrapping_add(1) % self.entries.len();
}
}
Pending => {
idx = idx.wrapping_add(1) % self.entries.len();
}
}
}
// If the map is empty, then the stream is complete.
if self.entries.is_empty() {
Ready(None)
} else {
Pending
}
}
}
fn col_stats_union(
mut left: ColumnStatistics,
right: ColumnStatistics,
) -> ColumnStatistics {
left.distinct_count = None;
left.min_value = left
.min_value
.zip(right.min_value)
.map(|(a, b)| expressions::helpers::min(&a, &b))
.and_then(Result::ok);
left.max_value = left
.max_value
.zip(right.max_value)
.map(|(a, b)| expressions::helpers::max(&a, &b))
.and_then(Result::ok);
left.null_count = left.null_count.zip(right.null_count).map(|(a, b)| a + b);
left
}
fn stats_union(mut left: Statistics, right: Statistics) -> Statistics {
left.is_exact = left.is_exact && right.is_exact;
left.num_rows = left.num_rows.zip(right.num_rows).map(|(a, b)| a + b);
left.total_byte_size = left
.total_byte_size
.zip(right.total_byte_size)
.map(|(a, b)| a + b);
left.column_statistics =
left.column_statistics
.zip(right.column_statistics)
.map(|(a, b)| {
a.into_iter()
.zip(b)
.map(|(ca, cb)| col_stats_union(ca, cb))
.collect()
});
left
}
#[cfg(test)]
mod tests {
use super::*;
use crate::test;
use crate::prelude::SessionContext;
use crate::{physical_plan::collect, scalar::ScalarValue};
use arrow::record_batch::RecordBatch;
#[tokio::test]
async fn test_union_partitions() -> Result<()> {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
// Create csv's with different partitioning
let csv = test::scan_partitioned_csv(4)?;
let csv2 = test::scan_partitioned_csv(5)?;
let union_exec = Arc::new(UnionExec::new(vec![csv, csv2]));
// Should have 9 partitions and 9 output batches
assert_eq!(union_exec.output_partitioning().partition_count(), 9);
let result: Vec<RecordBatch> = collect(union_exec, task_ctx).await?;
assert_eq!(result.len(), 9);
Ok(())
}
#[tokio::test]
async fn test_stats_union() {
let left = Statistics {
is_exact: true,
num_rows: Some(5),
total_byte_size: Some(23),
column_statistics: Some(vec![
ColumnStatistics {
distinct_count: Some(5),
max_value: Some(ScalarValue::Int64(Some(21))),
min_value: Some(ScalarValue::Int64(Some(-4))),
null_count: Some(0),
},
ColumnStatistics {
distinct_count: Some(1),
max_value: Some(ScalarValue::Utf8(Some(String::from("x")))),
min_value: Some(ScalarValue::Utf8(Some(String::from("a")))),
null_count: Some(3),
},
ColumnStatistics {
distinct_count: None,
max_value: Some(ScalarValue::Float32(Some(1.1))),
min_value: Some(ScalarValue::Float32(Some(0.1))),
null_count: None,
},
]),
};
let right = Statistics {
is_exact: true,
num_rows: Some(7),
total_byte_size: Some(29),
column_statistics: Some(vec![
ColumnStatistics {
distinct_count: Some(3),
max_value: Some(ScalarValue::Int64(Some(34))),
min_value: Some(ScalarValue::Int64(Some(1))),
null_count: Some(1),
},
ColumnStatistics {
distinct_count: None,
max_value: Some(ScalarValue::Utf8(Some(String::from("c")))),
min_value: Some(ScalarValue::Utf8(Some(String::from("b")))),
null_count: None,
},
ColumnStatistics {
distinct_count: None,
max_value: None,
min_value: None,
null_count: None,
},
]),
};
let result = stats_union(left, right);
let expected = Statistics {
is_exact: true,
num_rows: Some(12),
total_byte_size: Some(52),
column_statistics: Some(vec![
ColumnStatistics {
distinct_count: None,
max_value: Some(ScalarValue::Int64(Some(34))),
min_value: Some(ScalarValue::Int64(Some(-4))),
null_count: Some(1),
},
ColumnStatistics {
distinct_count: None,
max_value: Some(ScalarValue::Utf8(Some(String::from("x")))),
min_value: Some(ScalarValue::Utf8(Some(String::from("a")))),
null_count: None,
},
ColumnStatistics {
distinct_count: None,
max_value: None,
min_value: None,
null_count: None,
},
]),
};
assert_eq!(result, expected);
}
}