@@ -24,34 +24,34 @@ use std::sync::Arc;
2424use std:: task:: { Context , Poll } ;
2525use std:: { any:: Any , vec} ;
2626
27+ use arrow:: array:: { ArrayRef , UInt64Builder } ;
28+ use arrow:: datatypes:: SchemaRef ;
29+ use arrow:: record_batch:: RecordBatch ;
30+ use futures:: stream:: Stream ;
31+ use futures:: { FutureExt , StreamExt } ;
32+ use hashbrown:: HashMap ;
33+ use log:: trace;
34+ use parking_lot:: Mutex ;
35+ use tokio:: task:: JoinHandle ;
36+
37+ use datafusion_common:: { not_impl_err, DataFusionError , Result } ;
38+ use datafusion_execution:: memory_pool:: MemoryConsumer ;
39+ use datafusion_execution:: TaskContext ;
40+ use datafusion_physical_expr:: { EquivalenceProperties , PhysicalExpr } ;
41+
2742use crate :: common:: transpose;
2843use crate :: hash_utils:: create_hashes;
2944use crate :: metrics:: BaselineMetrics ;
3045use crate :: repartition:: distributor_channels:: { channels, partition_aware_channels} ;
3146use crate :: sorts:: streaming_merge;
3247use crate :: { DisplayFormatType , ExecutionPlan , Partitioning , Statistics } ;
3348
34- use self :: distributor_channels:: { DistributionReceiver , DistributionSender } ;
35-
3649use super :: common:: { AbortOnDropMany , AbortOnDropSingle , SharedMemoryReservation } ;
3750use super :: expressions:: PhysicalSortExpr ;
3851use super :: metrics:: { self , ExecutionPlanMetricsSet , MetricBuilder , MetricsSet } ;
3952use super :: { DisplayAs , RecordBatchStream , SendableRecordBatchStream } ;
4053
41- use arrow:: array:: { ArrayRef , UInt64Builder } ;
42- use arrow:: datatypes:: SchemaRef ;
43- use arrow:: record_batch:: RecordBatch ;
44- use datafusion_common:: { not_impl_err, DataFusionError , Result } ;
45- use datafusion_execution:: memory_pool:: MemoryConsumer ;
46- use datafusion_execution:: TaskContext ;
47- use datafusion_physical_expr:: { EquivalenceProperties , PhysicalExpr } ;
48-
49- use futures:: stream:: Stream ;
50- use futures:: { FutureExt , StreamExt } ;
51- use hashbrown:: HashMap ;
52- use log:: trace;
53- use parking_lot:: Mutex ;
54- use tokio:: task:: JoinHandle ;
54+ use self :: distributor_channels:: { DistributionReceiver , DistributionSender } ;
5555
5656mod distributor_channels;
5757
@@ -428,9 +428,12 @@ impl ExecutionPlan for RepartitionExec {
428428 self : Arc < Self > ,
429429 mut children : Vec < Arc < dyn ExecutionPlan > > ,
430430 ) -> Result < Arc < dyn ExecutionPlan > > {
431- let repartition =
432- RepartitionExec :: try_new ( children. swap_remove ( 0 ) , self . partitioning . clone ( ) ) ;
433- repartition. map ( |r| Arc :: new ( r. with_preserve_order ( self . preserve_order ) ) as _ )
431+ let mut repartition =
432+ RepartitionExec :: try_new ( children. swap_remove ( 0 ) , self . partitioning . clone ( ) ) ?;
433+ if self . preserve_order {
434+ repartition = repartition. with_preserve_order ( ) ;
435+ }
436+ Ok ( Arc :: new ( repartition) )
434437 }
435438
436439 /// Specifies whether this plan generates an infinite stream of records.
@@ -628,7 +631,9 @@ impl ExecutionPlan for RepartitionExec {
628631}
629632
630633impl RepartitionExec {
631- /// Create a new RepartitionExec
634+ /// Create a new RepartitionExec, that produces output `partitioning`, and
635+ /// does not preserve the order of the input (see [`Self::with_preserve_order`]
636+ /// for more details)
632637 pub fn try_new (
633638 input : Arc < dyn ExecutionPlan > ,
634639 partitioning : Partitioning ,
@@ -652,8 +657,8 @@ impl RepartitionExec {
652657 ///
653658 /// If the input is not ordered, or has only one partition, this is a no op,
654659 /// and the node remains a `RepartitionExec`.
655- pub fn with_preserve_order ( mut self , preserve_order : bool ) -> Self {
656- self . preserve_order = preserve_order &&
660+ pub fn with_preserve_order ( mut self ) -> Self {
661+ self . preserve_order =
657662 // If the input isn't ordered, there is no ordering to preserve
658663 self . input . output_ordering ( ) . is_some ( ) &&
659664 // if there is only one input partition, merging is not required
@@ -918,7 +923,19 @@ impl RecordBatchStream for PerPartitionStream {
918923
919924#[ cfg( test) ]
920925mod tests {
921- use super :: * ;
926+ use std:: collections:: HashSet ;
927+
928+ use arrow:: array:: { ArrayRef , StringArray } ;
929+ use arrow:: datatypes:: { DataType , Field , Schema } ;
930+ use arrow:: record_batch:: RecordBatch ;
931+ use arrow_array:: UInt32Array ;
932+ use futures:: FutureExt ;
933+ use tokio:: task:: JoinHandle ;
934+
935+ use datafusion_common:: cast:: as_string_array;
936+ use datafusion_common:: { assert_batches_sorted_eq, exec_err} ;
937+ use datafusion_execution:: runtime_env:: { RuntimeConfig , RuntimeEnv } ;
938+
922939 use crate :: {
923940 test:: {
924941 assert_is_pending,
@@ -929,16 +946,8 @@ mod tests {
929946 } ,
930947 { collect, expressions:: col, memory:: MemoryExec } ,
931948 } ;
932- use arrow:: array:: { ArrayRef , StringArray } ;
933- use arrow:: datatypes:: { DataType , Field , Schema } ;
934- use arrow:: record_batch:: RecordBatch ;
935- use arrow_array:: UInt32Array ;
936- use datafusion_common:: cast:: as_string_array;
937- use datafusion_common:: { assert_batches_sorted_eq, exec_err} ;
938- use datafusion_execution:: runtime_env:: { RuntimeConfig , RuntimeEnv } ;
939- use futures:: FutureExt ;
940- use std:: collections:: HashSet ;
941- use tokio:: task:: JoinHandle ;
949+
950+ use super :: * ;
942951
943952 #[ tokio:: test]
944953 async fn one_to_many_round_robin ( ) -> Result < ( ) > {
@@ -1443,11 +1452,14 @@ mod tests {
14431452#[ cfg( test) ]
14441453mod test {
14451454 use arrow_schema:: { DataType , Field , Schema , SortOptions } ;
1455+
14461456 use datafusion_physical_expr:: expressions:: col;
1457+
14471458 use crate :: memory:: MemoryExec ;
1448- use super :: * ;
14491459 use crate :: union:: UnionExec ;
14501460
1461+ use super :: * ;
1462+
14511463 /// Asserts that the plan is as expected
14521464 ///
14531465 /// `$EXPECTED_PLAN_LINES`: input plan
@@ -1477,9 +1489,10 @@ mod test {
14771489 let source2 = sorted_memory_exec ( & schema, sort_exprs) ;
14781490 // output has multiple partitions, and is sorted
14791491 let union = UnionExec :: new ( vec ! [ source1, source2] ) ;
1480- let exec = RepartitionExec :: try_new ( Arc :: new ( union) , Partitioning :: RoundRobinBatch ( 10 ) )
1481- . unwrap ( )
1482- . with_preserve_order ( true ) ;
1492+ let exec =
1493+ RepartitionExec :: try_new ( Arc :: new ( union) , Partitioning :: RoundRobinBatch ( 10 ) )
1494+ . unwrap ( )
1495+ . with_preserve_order ( ) ;
14831496
14841497 // Repartition should preserve order
14851498 let expected_plan = [
@@ -1498,9 +1511,9 @@ mod test {
14981511 let sort_exprs = sort_exprs ( & schema) ;
14991512 let source = sorted_memory_exec ( & schema, sort_exprs) ;
15001513 // output is sorted, but has only a single partition, so no need to sort
1501- let exec = RepartitionExec :: try_new ( source, Partitioning :: RoundRobinBatch ( 10 ) )
1514+ let exec = RepartitionExec :: try_new ( source, Partitioning :: RoundRobinBatch ( 10 ) )
15021515 . unwrap ( )
1503- . with_preserve_order ( true ) ;
1516+ . with_preserve_order ( ) ;
15041517
15051518 // Repartition should not preserve order
15061519 let expected_plan = [
@@ -1518,9 +1531,10 @@ mod test {
15181531 let source2 = memory_exec ( & schema) ;
15191532 // output has multiple partitions, but is not sorted
15201533 let union = UnionExec :: new ( vec ! [ source1, source2] ) ;
1521- let exec = RepartitionExec :: try_new ( Arc :: new ( union) , Partitioning :: RoundRobinBatch ( 10 ) )
1522- . unwrap ( )
1523- . with_preserve_order ( true ) ;
1534+ let exec =
1535+ RepartitionExec :: try_new ( Arc :: new ( union) , Partitioning :: RoundRobinBatch ( 10 ) )
1536+ . unwrap ( )
1537+ . with_preserve_order ( ) ;
15241538
15251539 // Repartition should not preserve order, as there is no order to preserve
15261540 let expected_plan = [
@@ -1545,18 +1559,18 @@ mod test {
15451559 } ]
15461560 }
15471561
1548-
15491562 fn memory_exec ( schema : & SchemaRef ) -> Arc < dyn ExecutionPlan > {
15501563 Arc :: new ( MemoryExec :: try_new ( & [ vec ! [ ] ] , schema. clone ( ) , None ) . unwrap ( ) )
15511564 }
15521565
1553- fn sorted_memory_exec ( schema : & SchemaRef , sort_exprs : Vec < PhysicalSortExpr > ) -> Arc < dyn ExecutionPlan > {
1566+ fn sorted_memory_exec (
1567+ schema : & SchemaRef ,
1568+ sort_exprs : Vec < PhysicalSortExpr > ,
1569+ ) -> Arc < dyn ExecutionPlan > {
15541570 Arc :: new (
1555- MemoryExec :: try_new ( & [ vec ! [ ] ] , schema. clone ( ) , None ) . unwrap ( )
1556- . with_sort_information ( vec ! [ sort_exprs] )
1571+ MemoryExec :: try_new ( & [ vec ! [ ] ] , schema. clone ( ) , None )
1572+ . unwrap ( )
1573+ . with_sort_information ( vec ! [ sort_exprs] ) ,
15571574 )
15581575 }
1559-
1560-
1561-
1562- }
1576+ }
0 commit comments