datafusion_physical_optimizer/enforce_sorting/
mod.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! EnforceSorting optimizer rule inspects the physical plan with respect
19//! to local sorting requirements and does the following:
20//! - Adds a [`SortExec`] when a requirement is not met,
21//! - Removes an already-existing [`SortExec`] if it is possible to prove
22//!   that this sort is unnecessary
23//!
24//! The rule can work on valid *and* invalid physical plans with respect to
25//! sorting requirements, but always produces a valid physical plan in this sense.
26//!
27//! A non-realistic but easy to follow example for sort removals: Assume that we
28//! somehow get the fragment
29//!
30//! ```text
31//! SortExec: expr=[nullable_col@0 ASC]
32//!   SortExec: expr=[non_nullable_col@1 ASC]
33//! ```
34//!
35//! in the physical plan. The first sort is unnecessary since its result is overwritten
36//! by another [`SortExec`]. Therefore, this rule removes it from the physical plan.
37
38pub mod replace_with_order_preserving_variants;
39pub mod sort_pushdown;
40
41use std::sync::Arc;
42
43use crate::enforce_sorting::replace_with_order_preserving_variants::{
44    replace_with_order_preserving_variants, OrderPreservationContext,
45};
46use crate::enforce_sorting::sort_pushdown::{
47    assign_initial_requirements, pushdown_sorts, SortPushDown,
48};
49use crate::output_requirements::OutputRequirementExec;
50use crate::utils::{
51    add_sort_above, add_sort_above_with_check, is_coalesce_partitions, is_limit,
52    is_repartition, is_sort, is_sort_preserving_merge, is_union, is_window,
53};
54use crate::PhysicalOptimizerRule;
55
56use datafusion_common::config::ConfigOptions;
57use datafusion_common::plan_err;
58use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode};
59use datafusion_common::Result;
60use datafusion_physical_expr::{Distribution, Partitioning};
61use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement};
62use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec;
63use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec};
64use datafusion_physical_plan::repartition::RepartitionExec;
65use datafusion_physical_plan::sorts::partial_sort::PartialSortExec;
66use datafusion_physical_plan::sorts::sort::SortExec;
67use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec;
68use datafusion_physical_plan::tree_node::PlanContext;
69use datafusion_physical_plan::windows::{
70    get_best_fitting_window, BoundedWindowAggExec, WindowAggExec,
71};
72use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties, InputOrderMode};
73
74use itertools::izip;
75
76/// This rule inspects [`SortExec`]'s in the given physical plan in order to
77/// remove unnecessary sorts, and optimize sort performance across the plan.
78#[derive(Default, Debug)]
79pub struct EnforceSorting {}
80
81impl EnforceSorting {
82    #[allow(missing_docs)]
83    pub fn new() -> Self {
84        Self {}
85    }
86}
87
88/// This context object is used within the [`EnforceSorting`] rule to track the closest
89/// [`SortExec`] descendant(s) for every child of a plan. The data attribute
90/// stores whether the plan is a `SortExec` or is connected to a `SortExec`
91/// via its children.
92pub type PlanWithCorrespondingSort = PlanContext<bool>;
93
94/// For a given node, update the [`PlanContext.data`] attribute.
95///
96/// If the node is a `SortExec`, or any of the node's children are a `SortExec`,
97/// then set the attribute to true.
98///
99/// This requires a bottom-up traversal was previously performed, updating the
100/// children previously.
101fn update_sort_ctx_children_data(
102    mut node_and_ctx: PlanWithCorrespondingSort,
103    data: bool,
104) -> Result<PlanWithCorrespondingSort> {
105    // Update `child.data` for all children.
106    for child_node in node_and_ctx.children.iter_mut() {
107        let child_plan = &child_node.plan;
108        child_node.data = if is_sort(child_plan) {
109            // child is sort
110            true
111        } else if is_limit(child_plan) {
112            // There is no sort linkage for this path, it starts at a limit.
113            false
114        } else {
115            // If a descendent is a sort, and the child maintains the sort.
116            let is_spm = is_sort_preserving_merge(child_plan);
117            let required_orderings = child_plan.required_input_ordering();
118            let flags = child_plan.maintains_input_order();
119            // Add parent node to the tree if there is at least one child with
120            // a sort connection:
121            izip!(flags, required_orderings).any(|(maintains, required_ordering)| {
122                let propagates_ordering =
123                    (maintains && required_ordering.is_none()) || is_spm;
124                // `connected_to_sort` only returns the correct answer with bottom-up traversal
125                let connected_to_sort =
126                    child_node.children.iter().any(|child| child.data);
127                propagates_ordering && connected_to_sort
128            })
129        }
130    }
131
132    // set data attribute on current node
133    node_and_ctx.data = data;
134
135    Ok(node_and_ctx)
136}
137
138/// This object is used within the [`EnforceSorting`] rule to track the closest
139/// [`CoalescePartitionsExec`] descendant(s) for every child of a plan. The data
140/// attribute stores whether the plan is a `CoalescePartitionsExec` or is
141/// connected to a `CoalescePartitionsExec` via its children.
142///
143/// The tracker halts at each [`SortExec`] (where the SPM will act to replace the coalesce).
144///
145/// This requires a bottom-up traversal was previously performed, updating the
146/// children previously.
147pub type PlanWithCorrespondingCoalescePartitions = PlanContext<bool>;
148
149/// Discovers the linked Coalesce->Sort cascades.
150///
151/// This linkage is used in [`remove_bottleneck_in_subplan`] to selectively
152/// remove the linked coalesces in the subplan. Then afterwards, an SPM is added
153/// at the root of the subplan (just after the sort) in order to parallelize sorts.
154/// Refer to the [`parallelize_sorts`] for more details on sort parallelization.
155///
156/// Example of linked Coalesce->Sort:
157/// ```text
158/// SortExec ctx.data=false, to halt remove_bottleneck_in_subplan)
159///   ...nodes...   ctx.data=true (e.g. are linked in cascade)
160///     Coalesce  ctx.data=true (e.g. is a coalesce)
161/// ```
162///
163/// The link should not be continued (and the coalesce not removed) if the distribution
164/// is changed between the Coalesce->Sort cascade. Example:
165/// ```text
166/// SortExec ctx.data=false, to halt remove_bottleneck_in_subplan)
167///   AggregateExec  ctx.data=false, to stop the link
168///     ...nodes...   ctx.data=true (e.g. are linked in cascade)
169///       Coalesce  ctx.data=true (e.g. is a coalesce)
170/// ```
171fn update_coalesce_ctx_children(
172    coalesce_context: &mut PlanWithCorrespondingCoalescePartitions,
173) {
174    let children = &coalesce_context.children;
175    coalesce_context.data = if children.is_empty() {
176        // Plan has no children, it cannot be a `CoalescePartitionsExec`.
177        false
178    } else if is_coalesce_partitions(&coalesce_context.plan) {
179        // Initiate a connection:
180        true
181    } else {
182        children.iter().enumerate().any(|(idx, node)| {
183            // Only consider operators that don't require a single partition,
184            // and connected to some `CoalescePartitionsExec`:
185            node.data
186                && !matches!(
187                    coalesce_context.plan.required_input_distribution()[idx],
188                    Distribution::SinglePartition
189                )
190        })
191    };
192}
193
194/// Performs optimizations based upon a series of subrules.
195/// Refer to each subrule for detailed descriptions of the optimizations performed:
196/// Subrule application is ordering dependent.
197///
198/// Optimizer consists of 5 main parts which work sequentially
199/// 1. [`ensure_sorting`] Works down-to-top to be able to remove unnecessary [`SortExec`]s, [`SortPreservingMergeExec`]s
200///    add [`SortExec`]s if necessary by a requirement and adjusts window operators.
201/// 2. [`parallelize_sorts`] (Optional, depends on the `repartition_sorts` configuration)
202///    Responsible to identify and remove unnecessary partition unifier operators
203///    such as [`SortPreservingMergeExec`], [`CoalescePartitionsExec`] follows [`SortExec`]s does possible simplifications.
204/// 3. [`replace_with_order_preserving_variants()`] Replaces with alternative operators, for example can merge
205///    a [`SortExec`] and a [`CoalescePartitionsExec`] into one [`SortPreservingMergeExec`]
206///    or a [`SortExec`] + [`RepartitionExec`] combination into an order preserving [`RepartitionExec`]
207/// 4. [`sort_pushdown`] Works top-down. Responsible to push down sort operators as deep as possible in the plan.
208/// 5. `replace_with_partial_sort` Checks if it's possible to replace [`SortExec`]s with [`PartialSortExec`] operators
209impl PhysicalOptimizerRule for EnforceSorting {
210    fn optimize(
211        &self,
212        plan: Arc<dyn ExecutionPlan>,
213        config: &ConfigOptions,
214    ) -> Result<Arc<dyn ExecutionPlan>> {
215        let plan_requirements = PlanWithCorrespondingSort::new_default(plan);
216        // Execute a bottom-up traversal to enforce sorting requirements,
217        // remove unnecessary sorts, and optimize sort-sensitive operators:
218        let adjusted = plan_requirements.transform_up(ensure_sorting)?.data;
219        let new_plan = if config.optimizer.repartition_sorts {
220            let plan_with_coalesce_partitions =
221                PlanWithCorrespondingCoalescePartitions::new_default(adjusted.plan);
222            let parallel = plan_with_coalesce_partitions
223                .transform_up(parallelize_sorts)
224                .data()?;
225            parallel.plan
226        } else {
227            adjusted.plan
228        };
229
230        let plan_with_pipeline_fixer = OrderPreservationContext::new_default(new_plan);
231        let updated_plan = plan_with_pipeline_fixer
232            .transform_up(|plan_with_pipeline_fixer| {
233                replace_with_order_preserving_variants(
234                    plan_with_pipeline_fixer,
235                    false,
236                    true,
237                    config,
238                )
239            })
240            .data()?;
241        // Execute a top-down traversal to exploit sort push-down opportunities
242        // missed by the bottom-up traversal:
243        let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan);
244        assign_initial_requirements(&mut sort_pushdown);
245        let adjusted = pushdown_sorts(sort_pushdown)?;
246        adjusted
247            .plan
248            .transform_up(|plan| Ok(Transformed::yes(replace_with_partial_sort(plan)?)))
249            .data()
250    }
251
252    fn name(&self) -> &str {
253        "EnforceSorting"
254    }
255
256    fn schema_check(&self) -> bool {
257        true
258    }
259}
260
261/// Only interested with [`SortExec`]s and their unbounded children.
262/// If the plan is not a [`SortExec`] or its child is not unbounded, returns the original plan.
263/// Otherwise, by checking the requirement satisfaction searches for a replacement chance.
264/// If there's one replaces the [`SortExec`] plan with a [`PartialSortExec`]
265fn replace_with_partial_sort(
266    plan: Arc<dyn ExecutionPlan>,
267) -> Result<Arc<dyn ExecutionPlan>> {
268    let plan_any = plan.as_any();
269    let Some(sort_plan) = plan_any.downcast_ref::<SortExec>() else {
270        return Ok(plan);
271    };
272
273    // It's safe to get first child of the SortExec
274    let child = Arc::clone(sort_plan.children()[0]);
275    if !child.boundedness().is_unbounded() {
276        return Ok(plan);
277    }
278
279    // Here we're trying to find the common prefix for sorted columns that is required for the
280    // sort and already satisfied by the given ordering
281    let child_eq_properties = child.equivalence_properties();
282    let sort_exprs = sort_plan.expr().clone();
283
284    let mut common_prefix_length = 0;
285    while child_eq_properties
286        .ordering_satisfy(sort_exprs[0..common_prefix_length + 1].to_vec())?
287    {
288        common_prefix_length += 1;
289    }
290    if common_prefix_length > 0 {
291        return Ok(Arc::new(
292            PartialSortExec::new(
293                sort_exprs,
294                Arc::clone(sort_plan.input()),
295                common_prefix_length,
296            )
297            .with_preserve_partitioning(sort_plan.preserve_partitioning())
298            .with_fetch(sort_plan.fetch()),
299        ));
300    }
301    Ok(plan)
302}
303
304/// Transform [`CoalescePartitionsExec`] + [`SortExec`] cascades into [`SortExec`]
305/// + [`SortPreservingMergeExec`] cascades, as illustrated below.
306///
307/// A [`CoalescePartitionsExec`] + [`SortExec`] cascade combines partitions
308/// first, and then sorts:
309/// ```text
310///   ┌ ─ ─ ─ ─ ─ ┐
311///    ┌─┬─┬─┐
312///   ││B│A│D│... ├──┐
313///    └─┴─┴─┘       │
314///   └ ─ ─ ─ ─ ─ ┘  │  ┌────────────────────────┐   ┌ ─ ─ ─ ─ ─ ─ ┐   ┌────────┐    ┌ ─ ─ ─ ─ ─ ─ ─ ┐
315///    Partition 1   │  │        Coalesce        │    ┌─┬─┬─┬─┬─┐      │        │     ┌─┬─┬─┬─┬─┐
316///                  ├──▶(no ordering guarantees)│──▶││B│E│A│D│C│...───▶  Sort  ├───▶││A│B│C│D│E│... │
317///                  │  │                        │    └─┴─┴─┴─┴─┘      │        │     └─┴─┴─┴─┴─┘
318///   ┌ ─ ─ ─ ─ ─ ┐  │  └────────────────────────┘   └ ─ ─ ─ ─ ─ ─ ┘   └────────┘    └ ─ ─ ─ ─ ─ ─ ─ ┘
319///    ┌─┬─┐         │                                 Partition                       Partition
320///   ││E│C│ ...  ├──┘
321///    └─┴─┘
322///   └ ─ ─ ─ ─ ─ ┘
323///    Partition 2
324/// ```
325///
326///
327/// A [`SortExec`] + [`SortPreservingMergeExec`] cascade sorts each partition
328/// first, then merges partitions while preserving the sort:
329/// ```text
330///   ┌ ─ ─ ─ ─ ─ ┐   ┌────────┐   ┌ ─ ─ ─ ─ ─ ┐
331///    ┌─┬─┬─┐        │        │    ┌─┬─┬─┐
332///   ││B│A│D│... │──▶│  Sort  │──▶││A│B│D│... │──┐
333///    └─┴─┴─┘        │        │    └─┴─┴─┘       │
334///   └ ─ ─ ─ ─ ─ ┘   └────────┘   └ ─ ─ ─ ─ ─ ┘  │  ┌─────────────────────┐    ┌ ─ ─ ─ ─ ─ ─ ─ ┐
335///    Partition 1                  Partition 1   │  │                     │     ┌─┬─┬─┬─┬─┐
336///                                               ├──▶ SortPreservingMerge ├───▶││A│B│C│D│E│... │
337///                                               │  │                     │     └─┴─┴─┴─┴─┘
338///   ┌ ─ ─ ─ ─ ─ ┐   ┌────────┐   ┌ ─ ─ ─ ─ ─ ┐  │  └─────────────────────┘    └ ─ ─ ─ ─ ─ ─ ─ ┘
339///    ┌─┬─┐          │        │    ┌─┬─┐         │                               Partition
340///   ││E│C│ ...  │──▶│  Sort  ├──▶││C│E│ ...  │──┘
341///    └─┴─┘          │        │    └─┴─┘
342///   └ ─ ─ ─ ─ ─ ┘   └────────┘   └ ─ ─ ─ ─ ─ ┘
343///    Partition 2                  Partition 2
344/// ```
345///
346/// The latter [`SortExec`] + [`SortPreservingMergeExec`] cascade performs
347/// sorting first on a per-partition basis, thereby parallelizing the sort.
348///
349/// The outcome is that plans of the form
350/// ```text
351///      "SortExec: expr=\[a@0 ASC\]",
352///      "  ...nodes..."
353///      "    CoalescePartitionsExec",
354///      "      RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1",
355/// ```
356/// are transformed into
357/// ```text
358///      "SortPreservingMergeExec: \[a@0 ASC\]",
359///      "  SortExec: expr=\[a@0 ASC\]",
360///      "    ...nodes..."
361///      "      RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1",
362/// ```
363/// by following connections from [`CoalescePartitionsExec`]s to [`SortExec`]s.
364/// By performing sorting in parallel, we can increase performance in some
365/// scenarios.
366///
367/// This optimization requires that there are no nodes between the [`SortExec`]
368/// and the [`CoalescePartitionsExec`], which requires single partitioning. Do
369/// not parallelize when the following scenario occurs:
370/// ```text
371///      "SortExec: expr=\[a@0 ASC\]",
372///      "  ...nodes requiring single partitioning..."
373///      "    CoalescePartitionsExec",
374///      "      RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1",
375/// ```
376///
377/// **Steps**
378/// 1. Checks if the plan is either a [`SortExec`], a [`SortPreservingMergeExec`],
379///    or a [`CoalescePartitionsExec`]. Otherwise, does nothing.
380/// 2. If the plan is a [`SortExec`] or a final [`SortPreservingMergeExec`]
381///    (i.e. output partitioning is 1):
382///      - Check for [`CoalescePartitionsExec`] in children. If found, check if
383///        it can be removed (with possible [`RepartitionExec`]s). If so, remove
384///        (see `remove_bottleneck_in_subplan`).
385///      - If the plan is satisfying the ordering requirements, add a `SortExec`.
386///      - Add an SPM above the plan and return.
387/// 3. If the plan is a [`CoalescePartitionsExec`]:
388///      - Check if it can be removed (with possible [`RepartitionExec`]s).
389///        If so, remove (see `remove_bottleneck_in_subplan`).
390pub fn parallelize_sorts(
391    mut requirements: PlanWithCorrespondingCoalescePartitions,
392) -> Result<Transformed<PlanWithCorrespondingCoalescePartitions>> {
393    update_coalesce_ctx_children(&mut requirements);
394
395    if requirements.children.is_empty() || !requirements.children[0].data {
396        // We only take an action when the plan is either a `SortExec`, a
397        // `SortPreservingMergeExec` or a `CoalescePartitionsExec`, and they
398        // all have a single child. Therefore, if the first child has no
399        // connection, we can return immediately.
400        Ok(Transformed::no(requirements))
401    } else if (is_sort(&requirements.plan)
402        || is_sort_preserving_merge(&requirements.plan))
403        && requirements.plan.output_partitioning().partition_count() <= 1
404    {
405        // Take the initial sort expressions and requirements
406        let (sort_exprs, fetch) = get_sort_exprs(&requirements.plan)?;
407        let sort_reqs = LexRequirement::from(sort_exprs.clone());
408        let sort_exprs = sort_exprs.clone();
409
410        // If there is a connection between a `CoalescePartitionsExec` and a
411        // global sort that satisfy the requirements (i.e. intermediate
412        // executors don't require single partition), then we can replace
413        // the `CoalescePartitionsExec` + `SortExec` cascade with a `SortExec`
414        // + `SortPreservingMergeExec` cascade to parallelize sorting.
415        requirements = remove_bottleneck_in_subplan(requirements)?;
416        // We also need to remove the self node since `remove_corresponding_coalesce_in_sub_plan`
417        // deals with the children and their children and so on.
418        requirements = requirements.children.swap_remove(0);
419
420        requirements = add_sort_above_with_check(requirements, sort_reqs, fetch)?;
421
422        let spm =
423            SortPreservingMergeExec::new(sort_exprs, Arc::clone(&requirements.plan));
424        Ok(Transformed::yes(
425            PlanWithCorrespondingCoalescePartitions::new(
426                Arc::new(spm.with_fetch(fetch)),
427                false,
428                vec![requirements],
429            ),
430        ))
431    } else if is_coalesce_partitions(&requirements.plan) {
432        let fetch = requirements.plan.fetch();
433        // There is an unnecessary `CoalescePartitionsExec` in the plan.
434        // This will handle the recursive `CoalescePartitionsExec` plans.
435        requirements = remove_bottleneck_in_subplan(requirements)?;
436        // For the removal of self node which is also a `CoalescePartitionsExec`.
437        requirements = requirements.children.swap_remove(0);
438
439        Ok(Transformed::yes(
440            PlanWithCorrespondingCoalescePartitions::new(
441                Arc::new(
442                    CoalescePartitionsExec::new(Arc::clone(&requirements.plan))
443                        .with_fetch(fetch),
444                ),
445                false,
446                vec![requirements],
447            ),
448        ))
449    } else {
450        Ok(Transformed::yes(requirements))
451    }
452}
453
454/// This function enforces sorting requirements and makes optimizations without
455/// violating these requirements whenever possible. Requires a bottom-up traversal.
456///
457/// **Steps**
458/// 1. Analyze if there are any immediate removals of [`SortExec`]s. If so,
459///    removes them (see `analyze_immediate_sort_removal`).
460/// 2. For each child of the plan, if the plan requires an input ordering:
461///      - Checks if ordering is satisfied with the child. If not:
462///          - If the child has an output ordering, removes the unnecessary
463///            `SortExec`.
464///          - Adds sort above the child plan.
465///      - (Plan not requires input ordering)
466///          - Checks if the `SortExec` is neutralized in the plan. If so,
467///            removes it.
468/// 3. Check and modify window operator:
469///      - Checks if the plan is a window operator, and connected with a sort.
470///        If so, either tries to update the window definition or removes
471///        unnecessary [`SortExec`]s (see `adjust_window_sort_removal`).
472/// 4. Check and remove possibly unnecessary SPM:
473///       -  Checks if the plan is SPM and child 1 output partitions, if so
474///          decides this SPM is unnecessary and removes it from the plan.
475pub fn ensure_sorting(
476    mut requirements: PlanWithCorrespondingSort,
477) -> Result<Transformed<PlanWithCorrespondingSort>> {
478    requirements = update_sort_ctx_children_data(requirements, false)?;
479
480    // Perform naive analysis at the beginning -- remove already-satisfied sorts:
481    if requirements.children.is_empty() {
482        return Ok(Transformed::no(requirements));
483    }
484    let maybe_requirements = analyze_immediate_sort_removal(requirements)?;
485    requirements = if !maybe_requirements.transformed {
486        maybe_requirements.data
487    } else {
488        return Ok(maybe_requirements);
489    };
490
491    let plan = &requirements.plan;
492    let mut updated_children = vec![];
493    for (idx, (required_ordering, mut child)) in plan
494        .required_input_ordering()
495        .into_iter()
496        .zip(requirements.children.into_iter())
497        .enumerate()
498    {
499        let physical_ordering = child.plan.output_ordering();
500
501        if let Some(required) = required_ordering {
502            let eq_properties = child.plan.equivalence_properties();
503            let req = required.into_single();
504            if !eq_properties.ordering_satisfy_requirement(req.clone())? {
505                // Make sure we preserve the ordering requirements:
506                if physical_ordering.is_some() {
507                    child = update_child_to_remove_unnecessary_sort(idx, child, plan)?;
508                }
509                child = add_sort_above(
510                    child,
511                    req,
512                    plan.as_any()
513                        .downcast_ref::<OutputRequirementExec>()
514                        .map(|output| output.fetch())
515                        .unwrap_or(None),
516                );
517                child = update_sort_ctx_children_data(child, true)?;
518            }
519        } else if physical_ordering.is_none()
520            || !plan.maintains_input_order()[idx]
521            || is_union(plan)
522        {
523            // We have a `SortExec` whose effect may be neutralized by another
524            // order-imposing operator, remove this sort:
525            child = update_child_to_remove_unnecessary_sort(idx, child, plan)?;
526        }
527        updated_children.push(child);
528    }
529    requirements.children = updated_children;
530    requirements = requirements.update_plan_from_children()?;
531    // For window expressions, we can remove some sorts when we can
532    // calculate the result in reverse:
533    let child_node = &requirements.children[0];
534    if is_window(&requirements.plan) && child_node.data {
535        return adjust_window_sort_removal(requirements).map(Transformed::yes);
536    } else if is_sort_preserving_merge(&requirements.plan)
537        && child_node.plan.output_partitioning().partition_count() <= 1
538    {
539        // This `SortPreservingMergeExec` is unnecessary, input already has a
540        // single partition and no fetch is required.
541        let mut child_node = requirements.children.swap_remove(0);
542        if let Some(fetch) = requirements.plan.fetch() {
543            // Add the limit exec if the original SPM had a fetch:
544            child_node.plan =
545                Arc::new(LocalLimitExec::new(Arc::clone(&child_node.plan), fetch));
546        }
547        return Ok(Transformed::yes(child_node));
548    }
549    update_sort_ctx_children_data(requirements, false).map(Transformed::yes)
550}
551
552/// Analyzes if there are any immediate sort removals by checking the `SortExec`s
553/// and their ordering requirement satisfactions with children
554/// If the sort is unnecessary, either replaces it with [`SortPreservingMergeExec`]/`LimitExec`
555/// or removes the [`SortExec`].
556/// Otherwise, returns the original plan
557fn analyze_immediate_sort_removal(
558    mut node: PlanWithCorrespondingSort,
559) -> Result<Transformed<PlanWithCorrespondingSort>> {
560    let Some(sort_exec) = node.plan.as_any().downcast_ref::<SortExec>() else {
561        return Ok(Transformed::no(node));
562    };
563    let sort_input = sort_exec.input();
564    // Check if the sort is unnecessary:
565    let properties = sort_exec.properties();
566    if let Some(ordering) = properties.output_ordering().cloned() {
567        let eqp = sort_input.equivalence_properties();
568        if !eqp.ordering_satisfy(ordering)? {
569            return Ok(Transformed::no(node));
570        }
571    }
572    node.plan = if !sort_exec.preserve_partitioning()
573        && sort_input.output_partitioning().partition_count() > 1
574    {
575        // Replace the sort with a sort-preserving merge:
576        Arc::new(
577            SortPreservingMergeExec::new(
578                sort_exec.expr().clone(),
579                Arc::clone(sort_input),
580            )
581            .with_fetch(sort_exec.fetch()),
582        ) as _
583    } else {
584        // Remove the sort:
585        node.children = node.children.swap_remove(0).children;
586        if let Some(fetch) = sort_exec.fetch() {
587            // If the sort has a fetch, we need to add a limit:
588            if properties.output_partitioning().partition_count() == 1 {
589                Arc::new(GlobalLimitExec::new(Arc::clone(sort_input), 0, Some(fetch)))
590            } else {
591                Arc::new(LocalLimitExec::new(Arc::clone(sort_input), fetch))
592            }
593        } else {
594            Arc::clone(sort_input)
595        }
596    };
597    for child in node.children.iter_mut() {
598        child.data = false;
599    }
600    node.data = false;
601    Ok(Transformed::yes(node))
602}
603
604/// Adjusts a [`WindowAggExec`] or a [`BoundedWindowAggExec`] to determine
605/// whether it may allow removing a sort.
606fn adjust_window_sort_removal(
607    mut window_tree: PlanWithCorrespondingSort,
608) -> Result<PlanWithCorrespondingSort> {
609    // Window operators have a single child we need to adjust:
610    let child_node = remove_corresponding_sort_from_sub_plan(
611        window_tree.children.swap_remove(0),
612        matches!(
613            window_tree.plan.required_input_distribution()[0],
614            Distribution::SinglePartition
615        ),
616    )?;
617    window_tree.children.push(child_node);
618
619    let plan = window_tree.plan.as_any();
620    let child_plan = &window_tree.children[0].plan;
621    let (window_expr, new_window) =
622        if let Some(exec) = plan.downcast_ref::<WindowAggExec>() {
623            let window_expr = exec.window_expr();
624            let new_window =
625                get_best_fitting_window(window_expr, child_plan, &exec.partition_keys())?;
626            (window_expr, new_window)
627        } else if let Some(exec) = plan.downcast_ref::<BoundedWindowAggExec>() {
628            let window_expr = exec.window_expr();
629            let new_window =
630                get_best_fitting_window(window_expr, child_plan, &exec.partition_keys())?;
631            (window_expr, new_window)
632        } else {
633            return plan_err!("Expected WindowAggExec or BoundedWindowAggExec");
634        };
635
636    window_tree.plan = if let Some(new_window) = new_window {
637        // We were able to change the window to accommodate the input, use it:
638        new_window
639    } else {
640        // We were unable to change the window to accommodate the input, so we
641        // will insert a sort.
642        let reqs = window_tree.plan.required_input_ordering().swap_remove(0);
643
644        // Satisfy the ordering requirement so that the window can run:
645        let mut child_node = window_tree.children.swap_remove(0);
646        if let Some(reqs) = reqs {
647            child_node = add_sort_above(child_node, reqs.into_single(), None);
648        }
649        let child_plan = Arc::clone(&child_node.plan);
650        window_tree.children.push(child_node);
651
652        if window_expr.iter().all(|e| e.uses_bounded_memory()) {
653            Arc::new(BoundedWindowAggExec::try_new(
654                window_expr.to_vec(),
655                child_plan,
656                InputOrderMode::Sorted,
657                !window_expr[0].partition_by().is_empty(),
658            )?) as _
659        } else {
660            Arc::new(WindowAggExec::try_new(
661                window_expr.to_vec(),
662                child_plan,
663                !window_expr[0].partition_by().is_empty(),
664            )?) as _
665        }
666    };
667
668    window_tree.data = false;
669    Ok(window_tree)
670}
671
672/// Removes parallelization-reducing, avoidable [`CoalescePartitionsExec`]s from
673/// the plan in `node`. After the removal of such `CoalescePartitionsExec`s from
674/// the plan, some of the remaining `RepartitionExec`s might become unnecessary.
675/// Removes such `RepartitionExec`s from the plan as well.
676fn remove_bottleneck_in_subplan(
677    mut requirements: PlanWithCorrespondingCoalescePartitions,
678) -> Result<PlanWithCorrespondingCoalescePartitions> {
679    let plan = &requirements.plan;
680    let children = &mut requirements.children;
681    if is_coalesce_partitions(&children[0].plan) {
682        // We can safely use the 0th index since we have a `CoalescePartitionsExec`.
683        let mut new_child_node = children[0].children.swap_remove(0);
684        while new_child_node.plan.output_partitioning() == plan.output_partitioning()
685            && is_repartition(&new_child_node.plan)
686            && is_repartition(plan)
687        {
688            new_child_node = new_child_node.children.swap_remove(0)
689        }
690        children[0] = new_child_node;
691    } else {
692        requirements.children = requirements
693            .children
694            .into_iter()
695            .map(|node| {
696                if node.data {
697                    remove_bottleneck_in_subplan(node)
698                } else {
699                    Ok(node)
700                }
701            })
702            .collect::<Result<_>>()?;
703    }
704    let mut new_reqs = requirements.update_plan_from_children()?;
705    if let Some(repartition) = new_reqs.plan.as_any().downcast_ref::<RepartitionExec>() {
706        let input_partitioning = repartition.input().output_partitioning();
707        // We can remove this repartitioning operator if it is now a no-op:
708        let mut can_remove = input_partitioning.eq(repartition.partitioning());
709        // We can also remove it if we ended up with an ineffective RR:
710        if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning() {
711            can_remove |= *n_out == input_partitioning.partition_count();
712        }
713        if can_remove {
714            new_reqs = new_reqs.children.swap_remove(0)
715        }
716    }
717    Ok(new_reqs)
718}
719
720/// Updates child to remove the unnecessary sort below it.
721fn update_child_to_remove_unnecessary_sort(
722    child_idx: usize,
723    mut node: PlanWithCorrespondingSort,
724    parent: &Arc<dyn ExecutionPlan>,
725) -> Result<PlanWithCorrespondingSort> {
726    if node.data {
727        let requires_single_partition = matches!(
728            parent.required_input_distribution()[child_idx],
729            Distribution::SinglePartition
730        );
731        node = remove_corresponding_sort_from_sub_plan(node, requires_single_partition)?;
732    }
733    node.data = false;
734    Ok(node)
735}
736
737/// Removes the sort from the plan in `node`.
738fn remove_corresponding_sort_from_sub_plan(
739    mut node: PlanWithCorrespondingSort,
740    requires_single_partition: bool,
741) -> Result<PlanWithCorrespondingSort> {
742    // A `SortExec` is always at the bottom of the tree.
743    if let Some(sort_exec) = node.plan.as_any().downcast_ref::<SortExec>() {
744        // Do not remove sorts with fetch:
745        if sort_exec.fetch().is_none() {
746            node = node.children.swap_remove(0);
747        }
748    } else {
749        let mut any_connection = false;
750        let required_dist = node.plan.required_input_distribution();
751        node.children = node
752            .children
753            .into_iter()
754            .enumerate()
755            .map(|(idx, child)| {
756                if child.data {
757                    any_connection = true;
758                    remove_corresponding_sort_from_sub_plan(
759                        child,
760                        matches!(required_dist[idx], Distribution::SinglePartition),
761                    )
762                } else {
763                    Ok(child)
764                }
765            })
766            .collect::<Result<_>>()?;
767        node = node.update_plan_from_children()?;
768        if any_connection || node.children.is_empty() {
769            node = update_sort_ctx_children_data(node, false)?;
770        }
771
772        // Replace with variants that do not preserve order.
773        if is_sort_preserving_merge(&node.plan) {
774            node.children = node.children.swap_remove(0).children;
775            node.plan = Arc::clone(node.plan.children().swap_remove(0));
776        } else if let Some(repartition) =
777            node.plan.as_any().downcast_ref::<RepartitionExec>()
778        {
779            node.plan = Arc::new(RepartitionExec::try_new(
780                Arc::clone(&node.children[0].plan),
781                repartition.properties().output_partitioning().clone(),
782            )?) as _;
783        }
784    };
785    // Deleting a merging sort may invalidate distribution requirements.
786    // Ensure that we stay compliant with such requirements:
787    if requires_single_partition && node.plan.output_partitioning().partition_count() > 1
788    {
789        // If there is existing ordering, to preserve ordering use
790        // `SortPreservingMergeExec` instead of a `CoalescePartitionsExec`.
791        let plan = Arc::clone(&node.plan);
792        let fetch = plan.fetch();
793        let plan = if let Some(ordering) = plan.output_ordering() {
794            Arc::new(
795                SortPreservingMergeExec::new(ordering.clone(), plan).with_fetch(fetch),
796            ) as _
797        } else {
798            Arc::new(CoalescePartitionsExec::new(plan)) as _
799        };
800        node = PlanWithCorrespondingSort::new(plan, false, vec![node]);
801        node = update_sort_ctx_children_data(node, false)?;
802    }
803    Ok(node)
804}
805
806/// Converts an [ExecutionPlan] trait object to a [LexOrdering] reference when possible.
807fn get_sort_exprs(
808    sort_any: &Arc<dyn ExecutionPlan>,
809) -> Result<(&LexOrdering, Option<usize>)> {
810    if let Some(sort_exec) = sort_any.as_any().downcast_ref::<SortExec>() {
811        Ok((sort_exec.expr(), sort_exec.fetch()))
812    } else if let Some(spm) = sort_any.as_any().downcast_ref::<SortPreservingMergeExec>()
813    {
814        Ok((spm.expr(), spm.fetch()))
815    } else {
816        plan_err!("Given ExecutionPlan is not a SortExec or a SortPreservingMergeExec")
817    }
818}
819
820// Tests are in tests/cases/enforce_sorting.rs