@@ -401,7 +401,6 @@ mod tests {
401401 use crate :: DistributedExt ;
402402 use crate :: test_utils:: in_memory_channel_resolver:: InMemoryChannelResolver ;
403403 use crate :: test_utils:: parquet:: register_parquet_tables;
404- use crate :: test_utils:: test_task_estimator:: FixedDataSourceExecTaskEstimator ;
405404 use crate :: { assert_snapshot, display_plan_ascii} ;
406405 use datafusion:: error:: DataFusionError ;
407406 use datafusion:: execution:: SessionStateBuilder ;
@@ -458,13 +457,13 @@ mod tests {
458457 │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))]
459458 │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
460459 │ CoalesceBatchesExec: target_batch_size=8192
461- │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2
460+ │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3
462461 └──────────────────────────────────────────────────
463- ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7]
462+ ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7]
464463 │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4
465- │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
464+ │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
466465 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
467- │ PartitionIsolatorExec: t0:[p0,p1 ,__] t1:[__,__,p0]
466+ │ PartitionIsolatorExec: t0:[p0,__ ,__] t1:[__,p0,__] t2 :[__,__,p0]
468467 │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet
469468 └──────────────────────────────────────────────────
470469 " ) ;
@@ -487,13 +486,13 @@ mod tests {
487486 │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))]
488487 │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
489488 │ CoalesceBatchesExec: target_batch_size=8192
490- │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2
489+ │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3
491490 └──────────────────────────────────────────────────
492- ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7]
491+ ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7]
493492 │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4
494- │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
493+ │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
495494 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
496- │ PartitionIsolatorExec: t0:[p0,p1 ,__] t1:[__,__,p0]
495+ │ PartitionIsolatorExec: t0:[p0,__ ,__] t1:[__,p0,__] t2 :[__,__,p0]
497496 │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet
498497 └──────────────────────────────────────────────────
499498 " ) ;
@@ -550,30 +549,30 @@ mod tests {
550549 │ ProjectionExec: expr=[avg(weather.MaxTemp)@1 as MaxTemp, RainTomorrow@0 as RainTomorrow]
551550 │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MaxTemp)]
552551 │ CoalesceBatchesExec: target_batch_size=8192
553- │ [Stage 3] => NetworkShuffleExec: output_partitions=4, input_tasks=2
552+ │ [Stage 3] => NetworkShuffleExec: output_partitions=4, input_tasks=3
554553 └──────────────────────────────────────────────────
555554 ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3]
556555 │ ProjectionExec: expr=[avg(weather.MinTemp)@1 as MinTemp, RainTomorrow@0 as RainTomorrow]
557556 │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MinTemp)]
558557 │ CoalesceBatchesExec: target_batch_size=8192
559- │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2
558+ │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3
560559 └──────────────────────────────────────────────────
561- ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7]
560+ ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7]
562561 │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 8), input_partitions=4
563562 │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MinTemp)]
564563 │ CoalesceBatchesExec: target_batch_size=8192
565564 │ FilterExec: RainToday@1 = yes, projection=[MinTemp@0, RainTomorrow@2]
566- │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
567- │ PartitionIsolatorExec: t0:[p0,p1 ,__] t1:[__,__,p0]
565+ │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
566+ │ PartitionIsolatorExec: t0:[p0,__ ,__] t1:[__,p0,__] t2 :[__,__,p0]
568567 │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday, RainTomorrow], file_type=parquet, predicate=RainToday@1 = yes, pruning_predicate=RainToday_null_count@2 != row_count@3 AND RainToday_min@0 <= yes AND yes <= RainToday_max@1, required_guarantees=[RainToday in (yes)]
569568 └──────────────────────────────────────────────────
570- ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p0..p3]
569+ ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[p0..p3]
571570 │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 4), input_partitions=4
572571 │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MaxTemp)]
573572 │ CoalesceBatchesExec: target_batch_size=8192
574573 │ FilterExec: RainToday@1 = no, projection=[MaxTemp@0, RainTomorrow@2]
575- │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
576- │ PartitionIsolatorExec: t0:[p0,p1 ,__] t1:[__,__,p0]
574+ │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
575+ │ PartitionIsolatorExec: t0:[p0,__ ,__] t1:[__,p0,__] t2 :[__,__,p0]
577576 │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday, RainTomorrow], file_type=parquet, predicate=RainToday@1 = no, pruning_predicate=RainToday_null_count@2 != row_count@3 AND RainToday_min@0 <= no AND no <= RainToday_max@1, required_guarantees=[RainToday in (no)]
578577 └──────────────────────────────────────────────────
579578 " ) ;
@@ -588,11 +587,11 @@ mod tests {
588587 assert_snapshot ! ( plan, @r"
589588 ┌───── DistributedExec ── Tasks: t0:[p0]
590589 │ SortPreservingMergeExec: [MinTemp@0 DESC]
591- │ [Stage 1] => NetworkCoalesceExec: output_partitions=4 , input_tasks=2
590+ │ [Stage 1] => NetworkCoalesceExec: output_partitions=3 , input_tasks=3
592591 └──────────────────────────────────────────────────
593- ┌───── Stage 1 ── Tasks: t0:[p0..p1 ] t1:[p2..p3 ]
592+ ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2 ]
594593 │ SortExec: expr=[MinTemp@0 DESC], preserve_partitioning=[true]
595- │ PartitionIsolatorExec: t0:[p0,p1 ,__] t1:[__,__,p0]
594+ │ PartitionIsolatorExec: t0:[p0,__ ,__] t1:[__,p0,__] t2 :[__,__,p0]
596595 │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, MaxTemp, Rainfall, Evaporation, Sunshine, WindGustDir, WindGustSpeed, WindDir9am, WindDir3pm, WindSpeed9am, WindSpeed3pm, Humidity9am, Humidity3pm, Pressure9am, Pressure3pm, Cloud9am, Cloud3pm, Temp9am, Temp3pm, RainToday, RISK_MM, RainTomorrow], file_type=parquet
597596 └──────────────────────────────────────────────────
598597 " ) ;
@@ -612,13 +611,13 @@ mod tests {
612611 ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3]
613612 │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[]
614613 │ CoalesceBatchesExec: target_batch_size=8192
615- │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2
614+ │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3
616615 └──────────────────────────────────────────────────
617- ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7]
616+ ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7]
618617 │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4
619- │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
618+ │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
620619 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[]
621- │ PartitionIsolatorExec: t0:[p0,p1 ,__] t1:[__,__,p0]
620+ │ PartitionIsolatorExec: t0:[p0,__ ,__] t1:[__,p0,__] t2 :[__,__,p0]
622621 │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday, WindGustDir], file_type=parquet
623622 └──────────────────────────────────────────────────
624623 " ) ;
@@ -649,7 +648,6 @@ mod tests {
649648 . with_default_features ( )
650649 . with_config ( config)
651650 . with_distributed_execution ( InMemoryChannelResolver :: new ( ) )
652- . with_distributed_task_estimator ( FixedDataSourceExecTaskEstimator ( 2 ) )
653651 . build ( ) ;
654652
655653 let ctx = SessionContext :: new_with_state ( state) ;
0 commit comments