/Users/andrewlamb/Software/datafusion/datafusion/physical-plan/src/coalesce_partitions.rs
Line | Count | Source (jump to first uncovered line) |
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 | | //! Defines the merge plan for executing partitions in parallel and then merging the results |
19 | | //! into a single partition |
20 | | |
21 | | use std::any::Any; |
22 | | use std::sync::Arc; |
23 | | |
24 | | use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; |
25 | | use super::stream::{ObservedStream, RecordBatchReceiverStream}; |
26 | | use super::{ |
27 | | DisplayAs, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, |
28 | | Statistics, |
29 | | }; |
30 | | |
31 | | use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; |
32 | | |
33 | | use datafusion_common::{internal_err, Result}; |
34 | | use datafusion_execution::TaskContext; |
35 | | |
36 | | /// Merge execution plan executes partitions in parallel and combines them into a single |
37 | | /// partition. No guarantees are made about the order of the resulting partition. |
38 | | #[derive(Debug)] |
39 | | pub struct CoalescePartitionsExec { |
40 | | /// Input execution plan |
41 | | input: Arc<dyn ExecutionPlan>, |
42 | | /// Execution metrics |
43 | | metrics: ExecutionPlanMetricsSet, |
44 | | cache: PlanProperties, |
45 | | } |
46 | | |
47 | | impl CoalescePartitionsExec { |
48 | | /// Create a new CoalescePartitionsExec |
49 | 69 | pub fn new(input: Arc<dyn ExecutionPlan>) -> Self { |
50 | 69 | let cache = Self::compute_properties(&input); |
51 | 69 | CoalescePartitionsExec { |
52 | 69 | input, |
53 | 69 | metrics: ExecutionPlanMetricsSet::new(), |
54 | 69 | cache, |
55 | 69 | } |
56 | 69 | } |
57 | | |
58 | | /// Input execution plan |
59 | 0 | pub fn input(&self) -> &Arc<dyn ExecutionPlan> { |
60 | 0 | &self.input |
61 | 0 | } |
62 | | |
63 | | /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. |
64 | 69 | fn compute_properties(input: &Arc<dyn ExecutionPlan>) -> PlanProperties { |
65 | 69 | // Coalescing partitions loses existing orderings: |
66 | 69 | let mut eq_properties = input.equivalence_properties().clone(); |
67 | 69 | eq_properties.clear_orderings(); |
68 | 69 | eq_properties.clear_per_partition_constants(); |
69 | 69 | PlanProperties::new( |
70 | 69 | eq_properties, // Equivalence Properties |
71 | 69 | Partitioning::UnknownPartitioning(1), // Output Partitioning |
72 | 69 | input.execution_mode(), // Execution Mode |
73 | 69 | ) |
74 | 69 | } |
75 | | } |
76 | | |
77 | | impl DisplayAs for CoalescePartitionsExec { |
78 | 0 | fn fmt_as( |
79 | 0 | &self, |
80 | 0 | t: DisplayFormatType, |
81 | 0 | f: &mut std::fmt::Formatter, |
82 | 0 | ) -> std::fmt::Result { |
83 | 0 | match t { |
84 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
85 | 0 | write!(f, "CoalescePartitionsExec") |
86 | 0 | } |
87 | 0 | } |
88 | 0 | } |
89 | | } |
90 | | |
91 | | impl ExecutionPlan for CoalescePartitionsExec { |
92 | 0 | fn name(&self) -> &'static str { |
93 | 0 | "CoalescePartitionsExec" |
94 | 0 | } |
95 | | |
96 | | /// Return a reference to Any that can be used for downcasting |
97 | 0 | fn as_any(&self) -> &dyn Any { |
98 | 0 | self |
99 | 0 | } |
100 | | |
101 | 299 | fn properties(&self) -> &PlanProperties { |
102 | 299 | &self.cache |
103 | 299 | } |
104 | | |
105 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
106 | 0 | vec![&self.input] |
107 | 0 | } |
108 | | |
109 | 0 | fn benefits_from_input_partitioning(&self) -> Vec<bool> { |
110 | 0 | vec![false] |
111 | 0 | } |
112 | | |
113 | 0 | fn with_new_children( |
114 | 0 | self: Arc<Self>, |
115 | 0 | children: Vec<Arc<dyn ExecutionPlan>>, |
116 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
117 | 0 | Ok(Arc::new(CoalescePartitionsExec::new(Arc::clone( |
118 | 0 | &children[0], |
119 | 0 | )))) |
120 | 0 | } |
121 | | |
122 | 53 | fn execute( |
123 | 53 | &self, |
124 | 53 | partition: usize, |
125 | 53 | context: Arc<TaskContext>, |
126 | 53 | ) -> Result<SendableRecordBatchStream> { |
127 | 53 | // CoalescePartitionsExec produces a single partition |
128 | 53 | if 0 != partition { |
129 | 0 | return internal_err!("CoalescePartitionsExec invalid partition {partition}"); |
130 | 53 | } |
131 | 53 | |
132 | 53 | let input_partitions = self.input.output_partitioning().partition_count(); |
133 | 53 | match input_partitions { |
134 | 0 | 0 => internal_err!( |
135 | 0 | "CoalescePartitionsExec requires at least one input partition" |
136 | 0 | ), |
137 | | 1 => { |
138 | | // bypass any threading / metrics if there is a single partition |
139 | 16 | self.input.execute(0, context) |
140 | | } |
141 | | _ => { |
142 | 37 | let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); |
143 | 37 | // record the (very) minimal work done so that |
144 | 37 | // elapsed_compute is not reported as 0 |
145 | 37 | let elapsed_compute = baseline_metrics.elapsed_compute().clone(); |
146 | 37 | let _timer = elapsed_compute.timer(); |
147 | 37 | |
148 | 37 | // use a stream that allows each sender to put in at |
149 | 37 | // least one result in an attempt to maximize |
150 | 37 | // parallelism. |
151 | 37 | let mut builder = |
152 | 37 | RecordBatchReceiverStream::builder(self.schema(), input_partitions); |
153 | | |
154 | | // spawn independent tasks whose resulting streams (of batches) |
155 | | // are sent to the channel for consumption. |
156 | 230 | for part_i in 0..input_partitions37 { |
157 | 230 | builder.run_input( |
158 | 230 | Arc::clone(&self.input), |
159 | 230 | part_i, |
160 | 230 | Arc::clone(&context), |
161 | 230 | ); |
162 | 230 | } |
163 | | |
164 | 37 | let stream = builder.build(); |
165 | 37 | Ok(Box::pin(ObservedStream::new(stream, baseline_metrics))) |
166 | | } |
167 | | } |
168 | 53 | } |
169 | | |
170 | 0 | fn metrics(&self) -> Option<MetricsSet> { |
171 | 0 | Some(self.metrics.clone_inner()) |
172 | 0 | } |
173 | | |
174 | 16 | fn statistics(&self) -> Result<Statistics> { |
175 | 16 | self.input.statistics() |
176 | 16 | } |
177 | | |
178 | 0 | fn supports_limit_pushdown(&self) -> bool { |
179 | 0 | true |
180 | 0 | } |
181 | | } |
182 | | |
183 | | #[cfg(test)] |
184 | | mod tests { |
185 | | use super::*; |
186 | | use crate::test::exec::{ |
187 | | assert_strong_count_converges_to_zero, BlockingExec, PanicExec, |
188 | | }; |
189 | | use crate::test::{self, assert_is_pending}; |
190 | | use crate::{collect, common}; |
191 | | |
192 | | use arrow::datatypes::{DataType, Field, Schema}; |
193 | | |
194 | | use futures::FutureExt; |
195 | | |
196 | | #[tokio::test] |
197 | 1 | async fn merge() -> Result<()> { |
198 | 1 | let task_ctx = Arc::new(TaskContext::default()); |
199 | 1 | |
200 | 1 | let num_partitions = 4; |
201 | 1 | let csv = test::scan_partitioned(num_partitions); |
202 | 1 | |
203 | 1 | // input should have 4 partitions |
204 | 1 | assert_eq!(csv.output_partitioning().partition_count(), num_partitions); |
205 | 1 | |
206 | 1 | let merge = CoalescePartitionsExec::new(csv); |
207 | 1 | |
208 | 1 | // output of CoalescePartitionsExec should have a single partition |
209 | 1 | assert_eq!( |
210 | 1 | merge.properties().output_partitioning().partition_count(), |
211 | 1 | 1 |
212 | 1 | ); |
213 | 1 | |
214 | 1 | // the result should contain 4 batches (one per input partition) |
215 | 1 | let iter = merge.execute(0, task_ctx)?0 ; |
216 | 1 | let batches = common::collect(iter).await?0 ; |
217 | 1 | assert_eq!(batches.len(), num_partitions); |
218 | 1 | |
219 | 1 | // there should be a total of 400 rows (100 per each partition) |
220 | 4 | let row_count: usize = batches.iter().map(1 |batch| batch.num_rows()).sum(); |
221 | 1 | assert_eq!(row_count, 400); |
222 | 1 | |
223 | 1 | Ok(()) |
224 | 1 | } |
225 | | |
226 | | #[tokio::test] |
227 | 1 | async fn test_drop_cancel() -> Result<()> { |
228 | 1 | let task_ctx = Arc::new(TaskContext::default()); |
229 | 1 | let schema = |
230 | 1 | Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)])); |
231 | 1 | |
232 | 1 | let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 2)); |
233 | 1 | let refs = blocking_exec.refs(); |
234 | 1 | let coaelesce_partitions_exec = |
235 | 1 | Arc::new(CoalescePartitionsExec::new(blocking_exec)); |
236 | 1 | |
237 | 1 | let fut = collect(coaelesce_partitions_exec, task_ctx); |
238 | 1 | let mut fut = fut.boxed(); |
239 | 1 | |
240 | 1 | assert_is_pending(&mut fut); |
241 | 1 | drop(fut); |
242 | 1 | assert_strong_count_converges_to_zero(refs).await; |
243 | 1 | |
244 | 1 | Ok(()) |
245 | 1 | } |
246 | | |
247 | | #[tokio::test] |
248 | | #[should_panic(expected = "PanickingStream did panic")] |
249 | 1 | async fn test_panic() { |
250 | 1 | let task_ctx = Arc::new(TaskContext::default()); |
251 | 1 | let schema = |
252 | 1 | Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)])); |
253 | 1 | |
254 | 1 | let panicking_exec = Arc::new(PanicExec::new(Arc::clone(&schema), 2)); |
255 | 1 | let coalesce_partitions_exec = |
256 | 1 | Arc::new(CoalescePartitionsExec::new(panicking_exec)); |
257 | 1 | |
258 | 1 | collect(coalesce_partitions_exec, task_ctx).await.unwrap()0 ; |
259 | 1 | } |
260 | | } |