forked from datafusion-contrib/datafusion-distributed
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathdistributed_aggregation.rs
More file actions
244 lines (213 loc) · 12.4 KB
/
distributed_aggregation.rs
File metadata and controls
244 lines (213 loc) · 12.4 KB
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
#[cfg(all(feature = "integration", test))]
mod tests {
use datafusion::arrow::array::{Int32Array, StringArray};
use datafusion::arrow::datatypes::{DataType, Field, Schema};
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::arrow::util::pretty::pretty_format_batches;
use datafusion::physical_plan::{displayable, execute_stream};
use datafusion::prelude::SessionContext;
use datafusion_distributed::test_utils::localhost::start_localhost_context;
use datafusion_distributed::test_utils::parquet::register_parquet_tables;
use datafusion_distributed::test_utils::session_context::register_temp_parquet_table;
use datafusion_distributed::{DefaultSessionBuilder, assert_snapshot, display_plan_ascii};
use futures::TryStreamExt;
use std::error::Error;
use std::sync::Arc;
use uuid::Uuid;
#[tokio::test]
async fn distributed_aggregation() -> Result<(), Box<dyn Error>> {
let (ctx_distributed, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await;
let query =
r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*)"#;
let ctx = SessionContext::default();
*ctx.state_ref().write().config_mut() = ctx_distributed.copied_config();
register_parquet_tables(&ctx).await?;
let df = ctx.sql(query).await?;
let physical = df.create_physical_plan().await?;
let physical_str = displayable(physical.as_ref()).indent(true).to_string();
register_parquet_tables(&ctx_distributed).await?;
let df_distributed = ctx_distributed.sql(query).await?;
let physical_distributed = df_distributed.create_physical_plan().await?;
let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref(), false);
assert_snapshot!(physical_str,
@r"
ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday]
SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST]
SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true]
ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))]
AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
RepartitionExec: partitioning=Hash([RainToday@0], 3), input_partitions=3
AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
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
",
);
assert_snapshot!(physical_distributed_str,
@r"
┌───── DistributedExec ── Tasks: t0:[p0]
│ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday]
│ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST]
│ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2
└──────────────────────────────────────────────────
┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2]
│ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true]
│ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))]
│ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
│ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3
└──────────────────────────────────────────────────
┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5]
│ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1
│ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
│ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0]
│ 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
└──────────────────────────────────────────────────
",
);
let batches = pretty_format_batches(
&execute_stream(physical, ctx.task_ctx())?
.try_collect::<Vec<_>>()
.await?,
)?;
assert_snapshot!(batches, @r"
+----------+-----------+
| count(*) | RainToday |
+----------+-----------+
| 66 | Yes |
| 300 | No |
+----------+-----------+
");
let batches_distributed = pretty_format_batches(
&execute_stream(physical_distributed, ctx.task_ctx())?
.try_collect::<Vec<_>>()
.await?,
)?;
assert_snapshot!(batches_distributed, @r"
+----------+-----------+
| count(*) | RainToday |
+----------+-----------+
| 66 | Yes |
| 300 | No |
+----------+-----------+
");
Ok(())
}
#[tokio::test]
async fn distributed_aggregation_head_node_partitioned() -> Result<(), Box<dyn Error>> {
let (ctx_distributed, _guard, _) = start_localhost_context(6, DefaultSessionBuilder).await;
let query = r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday""#;
let ctx = SessionContext::default();
*ctx.state_ref().write().config_mut() = ctx_distributed.copied_config();
register_parquet_tables(&ctx).await?;
let df = ctx.sql(query).await?;
let physical = df.create_physical_plan().await?;
let physical_str = displayable(physical.as_ref()).indent(true).to_string();
register_parquet_tables(&ctx_distributed).await?;
let df_distributed = ctx_distributed.sql(query).await?;
let physical_distributed = df_distributed.create_physical_plan().await?;
let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref(), false);
assert_snapshot!(physical_str,
@r"
ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday]
AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
RepartitionExec: partitioning=Hash([RainToday@0], 3), input_partitions=3
AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
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
",
);
assert_snapshot!(physical_distributed_str,
@r"
┌───── DistributedExec ── Tasks: t0:[p0]
│ CoalescePartitionsExec
│ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2
└──────────────────────────────────────────────────
┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2]
│ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday]
│ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
│ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3
└──────────────────────────────────────────────────
┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5]
│ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1
│ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))]
│ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0]
│ 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
└──────────────────────────────────────────────────
",
);
Ok(())
}
/// Test that multiple first_value() aggregations work correctly in distributed queries.
// TODO: Once https://github.com/apache/datafusion/pull/18303 is merged, this test will lose
// meaning, since the PR above will mask the underlying problem. Different queries or
// a new approach must be used in this case.
#[tokio::test]
async fn test_multiple_first_value_aggregations() -> Result<(), Box<dyn Error>> {
let (ctx, _guard, _) = start_localhost_context(3, DefaultSessionBuilder).await;
let schema = Arc::new(Schema::new(vec![
Field::new("group_id", DataType::Int32, false),
Field::new("trace_id", DataType::Utf8, false),
Field::new("value", DataType::Int32, false),
]));
// Create 2 batches that will be stored as separate parquet files
let batch1 = RecordBatch::try_new(
schema.clone(),
vec![
Arc::new(Int32Array::from(vec![1, 2])),
Arc::new(StringArray::from(vec!["trace1", "trace2"])),
Arc::new(Int32Array::from(vec![100, 200])),
],
)?;
let batch2 = RecordBatch::try_new(
schema.clone(),
vec![
Arc::new(Int32Array::from(vec![3, 4])),
Arc::new(StringArray::from(vec!["trace3", "trace4"])),
Arc::new(Int32Array::from(vec![300, 400])),
],
)?;
let file1 =
register_temp_parquet_table("records_part1", schema.clone(), vec![batch1], &ctx)
.await?;
let file2 =
register_temp_parquet_table("records_part2", schema.clone(), vec![batch2], &ctx)
.await?;
// Create a partitioned table by registering multiple files
let temp_dir = std::env::temp_dir();
let table_dir = temp_dir.join(format!("partitioned_table_{}", Uuid::new_v4()));
std::fs::create_dir(&table_dir)?;
std::fs::copy(&file1, table_dir.join("part1.parquet"))?;
std::fs::copy(&file2, table_dir.join("part2.parquet"))?;
// Register the directory as a partitioned table
ctx.register_parquet(
"records_partitioned",
table_dir.to_str().unwrap(),
datafusion::prelude::ParquetReadOptions::default(),
)
.await?;
let query = r#"SELECT group_id, first_value(trace_id) AS fv1, first_value(value) AS fv2
FROM records_partitioned
GROUP BY group_id
ORDER BY group_id"#;
let df = ctx.sql(query).await?;
let physical = df.create_physical_plan().await?;
// Execute distributed query
let batches_distributed = execute_stream(physical, ctx.task_ctx())?
.try_collect::<Vec<_>>()
.await?;
let actual_result = pretty_format_batches(&batches_distributed)?;
let expected_result = "\
+----------+--------+-----+
| group_id | fv1 | fv2 |
+----------+--------+-----+
| 1 | trace1 | 100 |
| 2 | trace2 | 200 |
| 3 | trace3 | 300 |
| 4 | trace4 | 400 |
+----------+--------+-----+";
// Print them out, the error message from `assert_eq` is otherwise hard to read.
println!("{expected_result}");
println!("{actual_result}");
// Compare against result. The regression this is testing for would have NULL values in
// the second and third column.
assert_eq!(actual_result.to_string(), expected_result,);
Ok(())
}
}