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 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804
// 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.
//! Traits for physical query plan, supporting parallel execution for partitioned relations.
pub use self::metrics::Metric;
use self::metrics::MetricsSet;
use self::{
coalesce_partitions::CoalescePartitionsExec, display::DisplayableExecutionPlan,
};
use crate::physical_plan::expressions::PhysicalSortExpr;
use datafusion_common::Result;
pub use datafusion_common::{ColumnStatistics, Statistics};
use arrow::datatypes::SchemaRef;
use arrow::record_batch::RecordBatch;
use datafusion_common::utils::DataPtr;
pub use datafusion_expr::Accumulator;
pub use datafusion_expr::ColumnarValue;
use datafusion_physical_expr::equivalence::OrderingEquivalenceProperties;
pub use display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay};
use futures::stream::{Stream, TryStreamExt};
use std::fmt;
use std::fmt::Debug;
use tokio::task::JoinSet;
use datafusion_common::tree_node::Transformed;
use datafusion_common::DataFusionError;
use std::sync::Arc;
use std::task::{Context, Poll};
use std::{any::Any, pin::Pin};
/// Trait for types that stream [arrow::record_batch::RecordBatch]
pub trait RecordBatchStream: Stream<Item = Result<RecordBatch>> {
/// Returns the schema of this `RecordBatchStream`.
///
/// Implementation of this trait should guarantee that all `RecordBatch`'s returned by this
/// stream should have the same schema as returned from this method.
fn schema(&self) -> SchemaRef;
}
/// Trait for a [`Stream`] of [`RecordBatch`]es
pub type SendableRecordBatchStream = Pin<Box<dyn RecordBatchStream + Send>>;
/// EmptyRecordBatchStream can be used to create a RecordBatchStream
/// that will produce no results
pub struct EmptyRecordBatchStream {
/// Schema wrapped by Arc
schema: SchemaRef,
}
impl EmptyRecordBatchStream {
/// Create an empty RecordBatchStream
pub fn new(schema: SchemaRef) -> Self {
Self { schema }
}
}
impl RecordBatchStream for EmptyRecordBatchStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
impl Stream for EmptyRecordBatchStream {
type Item = Result<RecordBatch>;
fn poll_next(
self: Pin<&mut Self>,
_cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
Poll::Ready(None)
}
}
/// `ExecutionPlan` represent nodes in the DataFusion Physical Plan.
///
/// Each `ExecutionPlan` is partition-aware and is responsible for
/// creating the actual `async` [`SendableRecordBatchStream`]s
/// of [`RecordBatch`] that incrementally compute the operator's
/// output from its input partition.
///
/// [`ExecutionPlan`] can be displayed in a simplified form using the
/// return value from [`displayable`] in addition to the (normally
/// quite verbose) `Debug` output.
pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync {
/// Returns the execution plan as [`Any`](std::any::Any) so that it can be
/// downcast to a specific implementation.
fn as_any(&self) -> &dyn Any;
/// Get the schema for this execution plan
fn schema(&self) -> SchemaRef;
/// Specifies the output partitioning scheme of this plan
fn output_partitioning(&self) -> Partitioning;
/// 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(false)
}
/// If the output of this operator within each partition is sorted,
/// returns `Some(keys)` with the description of how it was sorted.
///
/// For example, Sort, (obviously) produces sorted output as does
/// SortPreservingMergeStream. Less obviously `Projection`
/// produces sorted output if its input was sorted as it does not
/// reorder the input rows,
///
/// It is safe to return `None` here if your operator does not
/// have any particular output order here
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]>;
/// Specifies the data distribution requirements for all the
/// children for this operator, By default it's [[Distribution::UnspecifiedDistribution]] for each child,
fn required_input_distribution(&self) -> Vec<Distribution> {
vec![Distribution::UnspecifiedDistribution; self.children().len()]
}
/// Specifies the ordering requirements for all of the children
/// For each child, it's the local ordering requirement within
/// each partition rather than the global ordering
///
/// NOTE that checking `!is_empty()` does **not** check for a
/// required input ordering. Instead, the correct check is that at
/// least one entry must be `Some`
fn required_input_ordering(&self) -> Vec<Option<Vec<PhysicalSortRequirement>>> {
vec![None; self.children().len()]
}
/// Returns `false` if this operator's implementation may reorder
/// rows within or between partitions.
///
/// For example, Projection, Filter, and Limit maintain the order
/// of inputs -- they may transform values (Projection) or not
/// produce the same number of rows that went in (Filter and
/// Limit), but the rows that are produced go in the same way.
///
/// DataFusion uses this metadata to apply certain optimizations
/// such as automatically repartitioning correctly.
///
/// The default implementation returns `false`
///
/// WARNING: if you override this default, you *MUST* ensure that
/// the operator's maintains the ordering invariant or else
/// DataFusion may produce incorrect results.
fn maintains_input_order(&self) -> Vec<bool> {
vec![false; self.children().len()]
}
/// Returns `true` if this operator would benefit from
/// partitioning its input (and thus from more parallelism). For
/// operators that do very little work the overhead of extra
/// parallelism may outweigh any benefits
///
/// The default implementation returns `true` unless this operator
/// has signalled it requires a single child input partition.
fn benefits_from_input_partitioning(&self) -> bool {
// By default try to maximize parallelism with more CPUs if
// possible
!self
.required_input_distribution()
.into_iter()
.any(|dist| matches!(dist, Distribution::SinglePartition))
}
/// Get the EquivalenceProperties within the plan
fn equivalence_properties(&self) -> EquivalenceProperties {
EquivalenceProperties::new(self.schema())
}
/// Get the OrderingEquivalenceProperties within the plan
fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties {
OrderingEquivalenceProperties::new(self.schema())
}
/// Get a list of child execution plans that provide the input for this plan. The returned list
/// will be empty for leaf nodes, will contain a single value for unary nodes, or two
/// values for binary nodes (such as joins).
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>>;
/// Returns a new plan where all children were replaced by new plans.
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>>;
/// creates an iterator
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream>;
/// Return a snapshot of the set of [`Metric`]s for this
/// [`ExecutionPlan`].
///
/// While the values of the metrics in the returned
/// [`MetricsSet`]s may change as execution progresses, the
/// specific metrics will not.
///
/// Once `self.execute()` has returned (technically the future is
/// resolved) for all available partitions, the set of metrics
/// should be complete. If this function is called prior to
/// `execute()` new metrics may appear in subsequent calls.
fn metrics(&self) -> Option<MetricsSet> {
None
}
/// Returns the global output statistics for this `ExecutionPlan` node.
fn statistics(&self) -> Statistics;
}
/// Indicate whether a data exchange is needed for the input of `plan`, which will be very helpful
/// especially for the distributed engine to judge whether need to deal with shuffling.
/// Currently there are 3 kinds of execution plan which needs data exchange
/// 1. RepartitionExec for changing the partition number between two operators
/// 2. CoalescePartitionsExec for collapsing all of the partitions into one without ordering guarantee
/// 3. SortPreservingMergeExec for collapsing all of the sorted partitions into one with ordering guarantee
pub fn need_data_exchange(plan: Arc<dyn ExecutionPlan>) -> bool {
if let Some(repart) = plan.as_any().downcast_ref::<RepartitionExec>() {
!matches!(
repart.output_partitioning(),
Partitioning::RoundRobinBatch(_)
)
} else if let Some(coalesce) = plan.as_any().downcast_ref::<CoalescePartitionsExec>()
{
coalesce.input().output_partitioning().partition_count() > 1
} else if let Some(sort_preserving_merge) =
plan.as_any().downcast_ref::<SortPreservingMergeExec>()
{
sort_preserving_merge
.input()
.output_partitioning()
.partition_count()
> 1
} else {
false
}
}
/// Returns a copy of this plan if we change any child according to the pointer comparison.
/// The size of `children` must be equal to the size of `ExecutionPlan::children()`.
pub fn with_new_children_if_necessary(
plan: Arc<dyn ExecutionPlan>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Transformed<Arc<dyn ExecutionPlan>>> {
let old_children = plan.children();
if children.len() != old_children.len() {
Err(DataFusionError::Internal(
"Wrong number of children".to_string(),
))
} else if children.is_empty()
|| children
.iter()
.zip(old_children.iter())
.any(|(c1, c2)| !Arc::data_ptr_eq(c1, c2))
{
Ok(Transformed::Yes(plan.with_new_children(children)?))
} else {
Ok(Transformed::No(plan))
}
}
/// Return a [wrapper](DisplayableExecutionPlan) around an
/// [`ExecutionPlan`] which can be displayed in various easier to
/// understand ways.
///
/// ```
/// use datafusion::prelude::*;
/// use datafusion::physical_plan::displayable;
/// use object_store::path::Path;
///
/// #[tokio::main]
/// async fn main() {
/// // Hard code target_partitions as it appears in the RepartitionExec output
/// let config = SessionConfig::new()
/// .with_target_partitions(3);
/// let mut ctx = SessionContext::with_config(config);
///
/// // register the a table
/// ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await.unwrap();
///
/// // create a plan to run a SQL query
/// let dataframe = ctx.sql("SELECT a FROM example WHERE a < 5").await.unwrap();
/// let physical_plan = dataframe.create_physical_plan().await.unwrap();
///
/// // Format using display string in verbose mode
/// let displayable_plan = displayable(physical_plan.as_ref());
/// let plan_string = format!("{}", displayable_plan.indent(true));
///
/// let working_directory = std::env::current_dir().unwrap();
/// let normalized = Path::from_filesystem_path(working_directory).unwrap();
/// let plan_string = plan_string.replace(normalized.as_ref(), "WORKING_DIR");
///
/// assert_eq!("CoalesceBatchesExec: target_batch_size=8192\
/// \n FilterExec: a@0 < 5\
/// \n RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1\
/// \n CsvExec: file_groups={1 group: [[WORKING_DIR/tests/data/example.csv]]}, projection=[a], has_header=true",
/// plan_string.trim());
///
/// let one_line = format!("{}", displayable_plan.one_line());
/// assert_eq!("CoalesceBatchesExec: target_batch_size=8192", one_line.trim());
/// }
/// ```
///
pub fn displayable(plan: &dyn ExecutionPlan) -> DisplayableExecutionPlan<'_> {
DisplayableExecutionPlan::new(plan)
}
/// Visit all children of this plan, according to the order defined on `ExecutionPlanVisitor`.
// Note that this would be really nice if it were a method on
// ExecutionPlan, but it can not be because it takes a generic
// parameter and `ExecutionPlan` is a trait
pub fn accept<V: ExecutionPlanVisitor>(
plan: &dyn ExecutionPlan,
visitor: &mut V,
) -> Result<(), V::Error> {
visitor.pre_visit(plan)?;
for child in plan.children() {
visit_execution_plan(child.as_ref(), visitor)?;
}
visitor.post_visit(plan)?;
Ok(())
}
/// Trait that implements the [Visitor
/// pattern](https://en.wikipedia.org/wiki/Visitor_pattern) for a
/// depth first walk of `ExecutionPlan` nodes. `pre_visit` is called
/// before any children are visited, and then `post_visit` is called
/// after all children have been visited.
////
/// To use, define a struct that implements this trait and then invoke
/// ['accept'].
///
/// For example, for an execution plan that looks like:
///
/// ```text
/// ProjectionExec: id
/// FilterExec: state = CO
/// CsvExec:
/// ```
///
/// The sequence of visit operations would be:
/// ```text
/// visitor.pre_visit(ProjectionExec)
/// visitor.pre_visit(FilterExec)
/// visitor.pre_visit(CsvExec)
/// visitor.post_visit(CsvExec)
/// visitor.post_visit(FilterExec)
/// visitor.post_visit(ProjectionExec)
/// ```
pub trait ExecutionPlanVisitor {
/// The type of error returned by this visitor
type Error;
/// Invoked on an `ExecutionPlan` plan before any of its child
/// inputs have been visited. If Ok(true) is returned, the
/// recursion continues. If Err(..) or Ok(false) are returned, the
/// recursion stops immediately and the error, if any, is returned
/// to `accept`
fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result<bool, Self::Error>;
/// Invoked on an `ExecutionPlan` plan *after* all of its child
/// inputs have been visited. The return value is handled the same
/// as the return value of `pre_visit`. The provided default
/// implementation returns `Ok(true)`.
fn post_visit(&mut self, _plan: &dyn ExecutionPlan) -> Result<bool, Self::Error> {
Ok(true)
}
}
/// Recursively calls `pre_visit` and `post_visit` for this node and
/// all of its children, as described on [`ExecutionPlanVisitor`]
pub fn visit_execution_plan<V: ExecutionPlanVisitor>(
plan: &dyn ExecutionPlan,
visitor: &mut V,
) -> Result<(), V::Error> {
visitor.pre_visit(plan)?;
for child in plan.children() {
visit_execution_plan(child.as_ref(), visitor)?;
}
visitor.post_visit(plan)?;
Ok(())
}
/// Execute the [ExecutionPlan] and collect the results in memory
pub async fn collect(
plan: Arc<dyn ExecutionPlan>,
context: Arc<TaskContext>,
) -> Result<Vec<RecordBatch>> {
let stream = execute_stream(plan, context)?;
common::collect(stream).await
}
/// Execute the [ExecutionPlan] and return a single stream of results
pub fn execute_stream(
plan: Arc<dyn ExecutionPlan>,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
match plan.output_partitioning().partition_count() {
0 => Ok(Box::pin(EmptyRecordBatchStream::new(plan.schema()))),
1 => plan.execute(0, context),
_ => {
// merge into a single partition
let plan = CoalescePartitionsExec::new(plan.clone());
// CoalescePartitionsExec must produce a single partition
assert_eq!(1, plan.output_partitioning().partition_count());
plan.execute(0, context)
}
}
}
/// Execute the [ExecutionPlan] and collect the results in memory
pub async fn collect_partitioned(
plan: Arc<dyn ExecutionPlan>,
context: Arc<TaskContext>,
) -> Result<Vec<Vec<RecordBatch>>> {
let streams = execute_stream_partitioned(plan, context)?;
let mut join_set = JoinSet::new();
// Execute the plan and collect the results into batches.
streams.into_iter().enumerate().for_each(|(idx, stream)| {
join_set.spawn(async move {
let result: Result<Vec<RecordBatch>> = stream.try_collect().await;
(idx, result)
});
});
let mut batches = vec![];
// Note that currently this doesn't identify the thread that panicked
//
// TODO: Replace with [join_next_with_id](https://docs.rs/tokio/latest/tokio/task/struct.JoinSet.html#method.join_next_with_id
// once it is stable
while let Some(result) = join_set.join_next().await {
match result {
Ok((idx, res)) => batches.push((idx, res?)),
Err(e) => {
if e.is_panic() {
std::panic::resume_unwind(e.into_panic());
} else {
unreachable!();
}
}
}
}
batches.sort_by_key(|(idx, _)| *idx);
let batches = batches.into_iter().map(|(_, batch)| batch).collect();
Ok(batches)
}
/// Execute the [ExecutionPlan] and return a vec with one stream per output partition
pub fn execute_stream_partitioned(
plan: Arc<dyn ExecutionPlan>,
context: Arc<TaskContext>,
) -> Result<Vec<SendableRecordBatchStream>> {
let num_partitions = plan.output_partitioning().partition_count();
let mut streams = Vec::with_capacity(num_partitions);
for i in 0..num_partitions {
streams.push(plan.execute(i, context.clone())?);
}
Ok(streams)
}
/// Partitioning schemes supported by operators.
#[derive(Debug, Clone)]
pub enum Partitioning {
/// Allocate batches using a round-robin algorithm and the specified number of partitions
RoundRobinBatch(usize),
/// Allocate rows based on a hash of one of more expressions and the specified number of
/// partitions
Hash(Vec<Arc<dyn PhysicalExpr>>, usize),
/// Unknown partitioning scheme with a known number of partitions
UnknownPartitioning(usize),
}
impl fmt::Display for Partitioning {
fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match self {
Partitioning::RoundRobinBatch(size) => write!(f, "RoundRobinBatch({size})"),
Partitioning::Hash(phy_exprs, size) => {
let phy_exprs_str = phy_exprs
.iter()
.map(|e| format!("{e}"))
.collect::<Vec<String>>()
.join(", ");
write!(f, "Hash([{phy_exprs_str}], {size})")
}
Partitioning::UnknownPartitioning(size) => {
write!(f, "UnknownPartitioning({size})")
}
}
}
}
impl Partitioning {
/// Returns the number of partitions in this partitioning scheme
pub fn partition_count(&self) -> usize {
use Partitioning::*;
match self {
RoundRobinBatch(n) | Hash(_, n) | UnknownPartitioning(n) => *n,
}
}
/// Returns true when the guarantees made by this [[Partitioning]] are sufficient to
/// satisfy the partitioning scheme mandated by the `required` [[Distribution]]
pub fn satisfy<F: FnOnce() -> EquivalenceProperties>(
&self,
required: Distribution,
equal_properties: F,
) -> bool {
match required {
Distribution::UnspecifiedDistribution => true,
Distribution::SinglePartition if self.partition_count() == 1 => true,
Distribution::HashPartitioned(required_exprs) => {
match self {
// Here we do not check the partition count for hash partitioning and assumes the partition count
// and hash functions in the system are the same. In future if we plan to support storage partition-wise joins,
// then we need to have the partition count and hash functions validation.
Partitioning::Hash(partition_exprs, _) => {
let fast_match =
expr_list_eq_strict_order(&required_exprs, partition_exprs);
// If the required exprs do not match, need to leverage the eq_properties provided by the child
// and normalize both exprs based on the eq_properties
if !fast_match {
let eq_properties = equal_properties();
let eq_classes = eq_properties.classes();
if !eq_classes.is_empty() {
let normalized_required_exprs = required_exprs
.iter()
.map(|e| {
normalize_expr_with_equivalence_properties(
e.clone(),
eq_classes,
)
})
.collect::<Vec<_>>();
let normalized_partition_exprs = partition_exprs
.iter()
.map(|e| {
normalize_expr_with_equivalence_properties(
e.clone(),
eq_classes,
)
})
.collect::<Vec<_>>();
expr_list_eq_strict_order(
&normalized_required_exprs,
&normalized_partition_exprs,
)
} else {
fast_match
}
} else {
fast_match
}
}
_ => false,
}
}
_ => false,
}
}
}
impl PartialEq for Partitioning {
fn eq(&self, other: &Partitioning) -> bool {
match (self, other) {
(
Partitioning::RoundRobinBatch(count1),
Partitioning::RoundRobinBatch(count2),
) if count1 == count2 => true,
(Partitioning::Hash(exprs1, count1), Partitioning::Hash(exprs2, count2))
if expr_list_eq_strict_order(exprs1, exprs2) && (count1 == count2) =>
{
true
}
_ => false,
}
}
}
/// Retrieves the ordering equivalence properties for a given schema and output ordering.
pub fn ordering_equivalence_properties_helper(
schema: SchemaRef,
eq_orderings: &[LexOrdering],
) -> OrderingEquivalenceProperties {
let mut oep = OrderingEquivalenceProperties::new(schema);
let first_ordering = if let Some(first) = eq_orderings.first() {
first
} else {
// Return an empty OrderingEquivalenceProperties:
return oep;
};
// First entry among eq_orderings is the head, skip it:
for ordering in eq_orderings.iter().skip(1) {
if !ordering.is_empty() {
oep.add_equal_conditions((first_ordering, ordering))
}
}
oep
}
/// Distribution schemes
#[derive(Debug, Clone)]
pub enum Distribution {
/// Unspecified distribution
UnspecifiedDistribution,
/// A single partition is required
SinglePartition,
/// Requires children to be distributed in such a way that the same
/// values of the keys end up in the same partition
HashPartitioned(Vec<Arc<dyn PhysicalExpr>>),
}
impl Distribution {
/// Creates a Partitioning for this Distribution to satisfy itself
pub fn create_partitioning(&self, partition_count: usize) -> Partitioning {
match self {
Distribution::UnspecifiedDistribution => {
Partitioning::UnknownPartitioning(partition_count)
}
Distribution::SinglePartition => Partitioning::UnknownPartitioning(1),
Distribution::HashPartitioned(expr) => {
Partitioning::Hash(expr.clone(), partition_count)
}
}
}
}
use datafusion_physical_expr::expressions::Column;
pub use datafusion_physical_expr::window::WindowExpr;
use datafusion_physical_expr::{
expr_list_eq_strict_order, normalize_expr_with_equivalence_properties, LexOrdering,
};
pub use datafusion_physical_expr::{AggregateExpr, PhysicalExpr};
use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement};
/// Applies an optional projection to a [`SchemaRef`], returning the
/// projected schema
///
/// Example:
/// ```
/// use arrow::datatypes::{SchemaRef, Schema, Field, DataType};
/// use datafusion::physical_plan::project_schema;
///
/// // Schema with columns 'a', 'b', and 'c'
/// let schema = SchemaRef::new(Schema::new(vec![
/// Field::new("a", DataType::Int32, true),
/// Field::new("b", DataType::Int64, true),
/// Field::new("c", DataType::Utf8, true),
/// ]));
///
/// // Pick columns 'c' and 'b'
/// let projection = Some(vec![2,1]);
/// let projected_schema = project_schema(
/// &schema,
/// projection.as_ref()
/// ).unwrap();
///
/// let expected_schema = SchemaRef::new(Schema::new(vec![
/// Field::new("c", DataType::Utf8, true),
/// Field::new("b", DataType::Int64, true),
/// ]));
///
/// assert_eq!(projected_schema, expected_schema);
/// ```
pub fn project_schema(
schema: &SchemaRef,
projection: Option<&Vec<usize>>,
) -> Result<SchemaRef> {
let schema = match projection {
Some(columns) => Arc::new(schema.project(columns)?),
None => Arc::clone(schema),
};
Ok(schema)
}
pub mod aggregates;
pub mod analyze;
pub mod coalesce_batches;
pub mod coalesce_partitions;
pub mod common;
pub mod display;
pub mod empty;
pub mod explain;
pub mod filter;
pub mod insert;
pub mod joins;
pub mod limit;
pub mod memory;
pub mod metrics;
pub mod projection;
pub mod repartition;
pub mod sorts;
pub mod stream;
pub mod streaming;
pub mod tree_node;
pub mod udaf;
pub mod union;
pub mod unnest;
pub mod values;
pub mod windows;
use crate::physical_plan::repartition::RepartitionExec;
use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec;
use datafusion_execution::TaskContext;
pub use datafusion_physical_expr::{expressions, functions, hash_utils, udf};
#[cfg(test)]
mod tests {
use super::*;
use arrow::datatypes::DataType;
use arrow::datatypes::Schema;
use crate::physical_plan::Distribution;
use crate::physical_plan::Partitioning;
use crate::physical_plan::PhysicalExpr;
use datafusion_physical_expr::expressions::Column;
use std::sync::Arc;
#[tokio::test]
async fn partitioning_satisfy_distribution() -> Result<()> {
let schema = Arc::new(Schema::new(vec![
arrow::datatypes::Field::new("column_1", DataType::Int64, false),
arrow::datatypes::Field::new("column_2", DataType::Utf8, false),
]));
let partition_exprs1: Vec<Arc<dyn PhysicalExpr>> = vec![
Arc::new(Column::new_with_schema("column_1", &schema).unwrap()),
Arc::new(Column::new_with_schema("column_2", &schema).unwrap()),
];
let partition_exprs2: Vec<Arc<dyn PhysicalExpr>> = vec![
Arc::new(Column::new_with_schema("column_2", &schema).unwrap()),
Arc::new(Column::new_with_schema("column_1", &schema).unwrap()),
];
let distribution_types = vec![
Distribution::UnspecifiedDistribution,
Distribution::SinglePartition,
Distribution::HashPartitioned(partition_exprs1.clone()),
];
let single_partition = Partitioning::UnknownPartitioning(1);
let unspecified_partition = Partitioning::UnknownPartitioning(10);
let round_robin_partition = Partitioning::RoundRobinBatch(10);
let hash_partition1 = Partitioning::Hash(partition_exprs1, 10);
let hash_partition2 = Partitioning::Hash(partition_exprs2, 10);
for distribution in distribution_types {
let result = (
single_partition.satisfy(distribution.clone(), || {
EquivalenceProperties::new(schema.clone())
}),
unspecified_partition.satisfy(distribution.clone(), || {
EquivalenceProperties::new(schema.clone())
}),
round_robin_partition.satisfy(distribution.clone(), || {
EquivalenceProperties::new(schema.clone())
}),
hash_partition1.satisfy(distribution.clone(), || {
EquivalenceProperties::new(schema.clone())
}),
hash_partition2.satisfy(distribution.clone(), || {
EquivalenceProperties::new(schema.clone())
}),
);
match distribution {
Distribution::UnspecifiedDistribution => {
assert_eq!(result, (true, true, true, true, true))
}
Distribution::SinglePartition => {
assert_eq!(result, (true, false, false, false, false))
}
Distribution::HashPartitioned(_) => {
assert_eq!(result, (false, false, false, true, false))
}
}
}
Ok(())
}
}