Skip to content

Commit 7c388ea

Browse files
committed
Fix remaining tests
1 parent e1f41df commit 7c388ea

File tree

3 files changed

+8
-8
lines changed

3 files changed

+8
-8
lines changed

src/distributed_physical_optimizer_rule.rs

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -425,7 +425,7 @@ mod tests {
425425
│ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4
426426
│ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
427427
│ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
428-
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__] t1:[__,__,p0,p1]
428+
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0]
429429
│ 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
430430
└──────────────────────────────────────────────────
431431
");
@@ -453,7 +453,7 @@ mod tests {
453453
│ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4
454454
│ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
455455
│ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
456-
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__] t1:[__,__,p0,p1]
456+
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0]
457457
│ 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
458458
└──────────────────────────────────────────────────
459459
");
@@ -524,7 +524,7 @@ mod tests {
524524
│ CoalesceBatchesExec: target_batch_size=8192
525525
│ FilterExec: RainToday@1 = yes, projection=[MinTemp@0, RainTomorrow@2]
526526
│ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
527-
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__] t1:[__,__,p0,p1]
527+
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0]
528528
│ 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)]
529529
└──────────────────────────────────────────────────
530530
┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3] t1:[p4,p5,p6,p7]
@@ -533,7 +533,7 @@ mod tests {
533533
│ CoalesceBatchesExec: target_batch_size=8192
534534
│ FilterExec: RainToday@1 = no, projection=[MaxTemp@0, RainTomorrow@2]
535535
│ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
536-
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__] t1:[__,__,p0,p1]
536+
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0]
537537
│ 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)]
538538
└──────────────────────────────────────────────────
539539
");
@@ -550,7 +550,7 @@ mod tests {
550550
└──────────────────────────────────────────────────
551551
┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3]
552552
│ SortExec: expr=[MinTemp@0 DESC], preserve_partitioning=[true]
553-
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__] t1:[__,__,p0,p1]
553+
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0]
554554
│ 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
555555
└──────────────────────────────────────────────────
556556
");
@@ -570,7 +570,7 @@ mod tests {
570570
│ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 4), input_partitions=4
571571
│ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
572572
│ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[]
573-
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__] t1:[__,__,p0,p1]
573+
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0]
574574
│ 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
575575
└──────────────────────────────────────────────────
576576
");

src/protobuf/distributed_codec.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2,8 +2,8 @@ use super::get_distributed_user_codec;
22
use crate::common::ComposedPhysicalExtensionCodec;
33
use crate::execution_plans::{NetworkCoalesceExec, NetworkCoalesceReady, NetworkShuffleReadyExec};
44
use crate::{NetworkShuffleExec, PartitionIsolatorExec};
5-
use arrow::datatypes::SchemaRef;
65
use datafusion::arrow::datatypes::Schema;
6+
use datafusion::arrow::datatypes::SchemaRef;
77
use datafusion::execution::FunctionRegistry;
88
use datafusion::physical_expr::EquivalenceProperties;
99
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};

tests/distributed_aggregation.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ mod tests {
5959
┌───── Stage 1 Tasks: t0:[p0,p1,p2] t1:[p3,p4,p5]
6060
│ RepartitionExec: partitioning=Hash([RainToday@0], 3), input_partitions=2
6161
│ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
62-
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__] t1:[__,__,p0,p1]
62+
│ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0]
6363
│ 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
6464
└──────────────────────────────────────────────────
6565
",

0 commit comments

Comments
 (0)