/Users/andrewlamb/Software/datafusion/datafusion/physical-plan/src/test/exec.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 | | //! Simple iterator over batches for use in testing |
19 | | |
20 | | use std::{ |
21 | | any::Any, |
22 | | pin::Pin, |
23 | | sync::{Arc, Weak}, |
24 | | task::{Context, Poll}, |
25 | | }; |
26 | | |
27 | | use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; |
28 | | use crate::{ |
29 | | common, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, |
30 | | PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, |
31 | | }; |
32 | | |
33 | | use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; |
34 | | use arrow::record_batch::RecordBatch; |
35 | | use datafusion_common::{internal_err, DataFusionError, Result}; |
36 | | use datafusion_execution::TaskContext; |
37 | | use datafusion_physical_expr::EquivalenceProperties; |
38 | | |
39 | | use futures::Stream; |
40 | | use tokio::sync::Barrier; |
41 | | |
42 | | /// Index into the data that has been returned so far |
43 | | #[derive(Debug, Default, Clone)] |
44 | | pub struct BatchIndex { |
45 | | inner: Arc<std::sync::Mutex<usize>>, |
46 | | } |
47 | | |
48 | | impl BatchIndex { |
49 | | /// Return the current index |
50 | 17 | pub fn value(&self) -> usize { |
51 | 17 | let inner = self.inner.lock().unwrap(); |
52 | 17 | *inner |
53 | 17 | } |
54 | | |
55 | | // increment the current index by one |
56 | 4 | pub fn incr(&self) { |
57 | 4 | let mut inner = self.inner.lock().unwrap(); |
58 | 4 | *inner += 1; |
59 | 4 | } |
60 | | } |
61 | | |
62 | | /// Iterator over batches |
63 | | #[derive(Debug, Default)] |
64 | | pub struct TestStream { |
65 | | /// Vector of record batches |
66 | | data: Vec<RecordBatch>, |
67 | | /// Index into the data that has been returned so far |
68 | | index: BatchIndex, |
69 | | } |
70 | | |
71 | | impl TestStream { |
72 | | /// Create an iterator for a vector of record batches. Assumes at |
73 | | /// least one entry in data (for the schema) |
74 | 3 | pub fn new(data: Vec<RecordBatch>) -> Self { |
75 | 3 | Self { |
76 | 3 | data, |
77 | 3 | ..Default::default() |
78 | 3 | } |
79 | 3 | } |
80 | | |
81 | | /// Return a handle to the index counter for this stream |
82 | 3 | pub fn index(&self) -> BatchIndex { |
83 | 3 | self.index.clone() |
84 | 3 | } |
85 | | } |
86 | | |
87 | | impl Stream for TestStream { |
88 | | type Item = Result<RecordBatch>; |
89 | | |
90 | 4 | fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll<Option<Self::Item>> { |
91 | 4 | let next_batch = self.index.value(); |
92 | 4 | |
93 | 4 | Poll::Ready(if next_batch < self.data.len() { |
94 | 4 | let next_batch = self.index.value(); |
95 | 4 | self.index.incr(); |
96 | 4 | Some(Ok(self.data[next_batch].clone())) |
97 | | } else { |
98 | 0 | None |
99 | | }) |
100 | 4 | } |
101 | | |
102 | 0 | fn size_hint(&self) -> (usize, Option<usize>) { |
103 | 0 | (self.data.len(), Some(self.data.len())) |
104 | 0 | } |
105 | | } |
106 | | |
107 | | impl RecordBatchStream for TestStream { |
108 | | /// Get the schema |
109 | 3 | fn schema(&self) -> SchemaRef { |
110 | 3 | self.data[0].schema() |
111 | 3 | } |
112 | | } |
113 | | |
114 | | /// A Mock ExecutionPlan that can be used for writing tests of other |
115 | | /// ExecutionPlans |
116 | | #[derive(Debug)] |
117 | | pub struct MockExec { |
118 | | /// the results to send back |
119 | | data: Vec<Result<RecordBatch>>, |
120 | | schema: SchemaRef, |
121 | | /// if true (the default), sends data using a separate task to ensure the |
122 | | /// batches are not available without this stream yielding first |
123 | | use_task: bool, |
124 | | cache: PlanProperties, |
125 | | } |
126 | | |
127 | | impl MockExec { |
128 | | /// Create a new `MockExec` with a single partition that returns |
129 | | /// the specified `Results`s. |
130 | | /// |
131 | | /// By default, the batches are not produced immediately (the |
132 | | /// caller has to actually yield and another task must run) to |
133 | | /// ensure any poll loops are correct. This behavior can be |
134 | | /// changed with `with_use_task` |
135 | 6 | pub fn new(data: Vec<Result<RecordBatch>>, schema: SchemaRef) -> Self { |
136 | 6 | let cache = Self::compute_properties(Arc::clone(&schema)); |
137 | 6 | Self { |
138 | 6 | data, |
139 | 6 | schema, |
140 | 6 | use_task: true, |
141 | 6 | cache, |
142 | 6 | } |
143 | 6 | } |
144 | | |
145 | | /// If `use_task` is true (the default) then the batches are sent |
146 | | /// back using a separate task to ensure the underlying stream is |
147 | | /// not immediately ready |
148 | 1 | pub fn with_use_task(mut self, use_task: bool) -> Self { |
149 | 1 | self.use_task = use_task; |
150 | 1 | self |
151 | 1 | } |
152 | | |
153 | | /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. |
154 | 6 | fn compute_properties(schema: SchemaRef) -> PlanProperties { |
155 | 6 | let eq_properties = EquivalenceProperties::new(schema); |
156 | 6 | |
157 | 6 | PlanProperties::new( |
158 | 6 | eq_properties, |
159 | 6 | Partitioning::UnknownPartitioning(1), |
160 | 6 | ExecutionMode::Bounded, |
161 | 6 | ) |
162 | 6 | } |
163 | | } |
164 | | |
165 | | impl DisplayAs for MockExec { |
166 | 0 | fn fmt_as( |
167 | 0 | &self, |
168 | 0 | t: DisplayFormatType, |
169 | 0 | f: &mut std::fmt::Formatter, |
170 | 0 | ) -> std::fmt::Result { |
171 | 0 | match t { |
172 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
173 | 0 | write!(f, "MockExec") |
174 | 0 | } |
175 | 0 | } |
176 | 0 | } |
177 | | } |
178 | | |
179 | | impl ExecutionPlan for MockExec { |
180 | 0 | fn name(&self) -> &'static str { |
181 | 0 | Self::static_name() |
182 | 0 | } |
183 | | |
184 | 0 | fn as_any(&self) -> &dyn Any { |
185 | 0 | self |
186 | 0 | } |
187 | | |
188 | 106 | fn properties(&self) -> &PlanProperties { |
189 | 106 | &self.cache |
190 | 106 | } |
191 | | |
192 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
193 | 0 | vec![] |
194 | 0 | } |
195 | | |
196 | 0 | fn with_new_children( |
197 | 0 | self: Arc<Self>, |
198 | 0 | _: Vec<Arc<dyn ExecutionPlan>>, |
199 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
200 | 0 | unimplemented!() |
201 | | } |
202 | | |
203 | | /// Returns a stream which yields data |
204 | 12 | fn execute( |
205 | 12 | &self, |
206 | 12 | partition: usize, |
207 | 12 | _context: Arc<TaskContext>, |
208 | 12 | ) -> Result<SendableRecordBatchStream> { |
209 | 12 | assert_eq!(partition, 0); |
210 | | |
211 | | // Result doesn't implement clone, so do it ourself |
212 | 12 | let data: Vec<_> = self |
213 | 12 | .data |
214 | 12 | .iter() |
215 | 23 | .map(|r| match r { |
216 | 12 | Ok(batch) => Ok(batch.clone()), |
217 | 11 | Err(e) => Err(clone_error(e)), |
218 | 23 | }) |
219 | 12 | .collect(); |
220 | 12 | |
221 | 12 | if self.use_task { |
222 | 11 | let mut builder = RecordBatchReceiverStream::builder(self.schema(), 2); |
223 | 11 | // send data in order but in a separate task (to ensure |
224 | 11 | // the batches are not available without the stream |
225 | 11 | // yielding). |
226 | 11 | let tx = builder.tx(); |
227 | 11 | builder.spawn(async move { |
228 | 32 | for batch21 in data { |
229 | 21 | println!("Sending batch via delayed stream"); |
230 | 21 | if let Err(e0 ) = tx.send(batch).await0 { |
231 | 0 | println!("ERROR batch via delayed stream: {e}"); |
232 | 21 | } |
233 | | } |
234 | | |
235 | 11 | Ok(()) |
236 | 11 | }); |
237 | 11 | // returned stream simply reads off the rx stream |
238 | 11 | Ok(builder.build()) |
239 | | } else { |
240 | | // make an input that will error |
241 | 1 | let stream = futures::stream::iter(data); |
242 | 1 | Ok(Box::pin(RecordBatchStreamAdapter::new( |
243 | 1 | self.schema(), |
244 | 1 | stream, |
245 | 1 | ))) |
246 | | } |
247 | 12 | } |
248 | | |
249 | | // Panics if one of the batches is an error |
250 | 0 | fn statistics(&self) -> Result<Statistics> { |
251 | 0 | let data: Result<Vec<_>> = self |
252 | 0 | .data |
253 | 0 | .iter() |
254 | 0 | .map(|r| match r { |
255 | 0 | Ok(batch) => Ok(batch.clone()), |
256 | 0 | Err(e) => Err(clone_error(e)), |
257 | 0 | }) |
258 | 0 | .collect(); |
259 | | |
260 | 0 | let data = data?; |
261 | | |
262 | 0 | Ok(common::compute_record_batch_statistics( |
263 | 0 | &[data], |
264 | 0 | &self.schema, |
265 | 0 | None, |
266 | 0 | )) |
267 | 0 | } |
268 | | } |
269 | | |
270 | 11 | fn clone_error(e: &DataFusionError) -> DataFusionError { |
271 | | use DataFusionError::*; |
272 | 11 | match e { |
273 | 11 | Execution(msg) => Execution(msg.to_string()), |
274 | 0 | _ => unimplemented!(), |
275 | | } |
276 | 11 | } |
277 | | |
278 | | /// A Mock ExecutionPlan that does not start producing input until a |
279 | | /// barrier is called |
280 | | /// |
281 | | #[derive(Debug)] |
282 | | pub struct BarrierExec { |
283 | | /// partitions to send back |
284 | | data: Vec<Vec<RecordBatch>>, |
285 | | schema: SchemaRef, |
286 | | |
287 | | /// all streams wait on this barrier to produce |
288 | | barrier: Arc<Barrier>, |
289 | | cache: PlanProperties, |
290 | | } |
291 | | |
292 | | impl BarrierExec { |
293 | | /// Create a new exec with some number of partitions. |
294 | 3 | pub fn new(data: Vec<Vec<RecordBatch>>, schema: SchemaRef) -> Self { |
295 | 3 | // wait for all streams and the input |
296 | 3 | let barrier = Arc::new(Barrier::new(data.len() + 1)); |
297 | 3 | let cache = Self::compute_properties(Arc::clone(&schema), &data); |
298 | 3 | Self { |
299 | 3 | data, |
300 | 3 | schema, |
301 | 3 | barrier, |
302 | 3 | cache, |
303 | 3 | } |
304 | 3 | } |
305 | | |
306 | | /// wait until all the input streams and this function is ready |
307 | 3 | pub async fn wait(&self) { |
308 | 3 | println!("BarrierExec::wait waiting on barrier"); |
309 | 3 | self.barrier.wait().await; |
310 | 3 | println!("BarrierExec::wait done waiting"); |
311 | 3 | } |
312 | | |
313 | | /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. |
314 | 3 | fn compute_properties( |
315 | 3 | schema: SchemaRef, |
316 | 3 | data: &[Vec<RecordBatch>], |
317 | 3 | ) -> PlanProperties { |
318 | 3 | let eq_properties = EquivalenceProperties::new(schema); |
319 | 3 | PlanProperties::new( |
320 | 3 | eq_properties, |
321 | 3 | Partitioning::UnknownPartitioning(data.len()), |
322 | 3 | ExecutionMode::Bounded, |
323 | 3 | ) |
324 | 3 | } |
325 | | } |
326 | | |
327 | | impl DisplayAs for BarrierExec { |
328 | 0 | fn fmt_as( |
329 | 0 | &self, |
330 | 0 | t: DisplayFormatType, |
331 | 0 | f: &mut std::fmt::Formatter, |
332 | 0 | ) -> std::fmt::Result { |
333 | 0 | match t { |
334 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
335 | 0 | write!(f, "BarrierExec") |
336 | 0 | } |
337 | 0 | } |
338 | 0 | } |
339 | | } |
340 | | |
341 | | impl ExecutionPlan for BarrierExec { |
342 | 0 | fn name(&self) -> &'static str { |
343 | 0 | Self::static_name() |
344 | 0 | } |
345 | | |
346 | 0 | fn as_any(&self) -> &dyn Any { |
347 | 0 | self |
348 | 0 | } |
349 | | |
350 | 27 | fn properties(&self) -> &PlanProperties { |
351 | 27 | &self.cache |
352 | 27 | } |
353 | | |
354 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
355 | 0 | unimplemented!() |
356 | | } |
357 | | |
358 | 0 | fn with_new_children( |
359 | 0 | self: Arc<Self>, |
360 | 0 | _: Vec<Arc<dyn ExecutionPlan>>, |
361 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
362 | 0 | unimplemented!() |
363 | | } |
364 | | |
365 | | /// Returns a stream which yields data |
366 | 6 | fn execute( |
367 | 6 | &self, |
368 | 6 | partition: usize, |
369 | 6 | _context: Arc<TaskContext>, |
370 | 6 | ) -> Result<SendableRecordBatchStream> { |
371 | 6 | assert!(partition < self.data.len()); |
372 | | |
373 | 6 | let mut builder = RecordBatchReceiverStream::builder(self.schema(), 2); |
374 | 6 | |
375 | 6 | // task simply sends data in order after barrier is reached |
376 | 6 | let data = self.data[partition].clone(); |
377 | 6 | let b = Arc::clone(&self.barrier); |
378 | 6 | let tx = builder.tx(); |
379 | 6 | builder.spawn(async move { |
380 | 6 | println!("Partition {partition} waiting on barrier"); |
381 | 6 | b.wait().await3 ; |
382 | 18 | for batch12 in data { |
383 | 12 | println!("Partition {partition} sending batch"); |
384 | 12 | if let Err(e0 ) = tx.send(Ok(batch)).await0 { |
385 | 0 | println!("ERROR batch via barrier stream stream: {e}"); |
386 | 12 | } |
387 | | } |
388 | | |
389 | 6 | Ok(()) |
390 | 6 | }); |
391 | 6 | |
392 | 6 | // returned stream simply reads off the rx stream |
393 | 6 | Ok(builder.build()) |
394 | 6 | } |
395 | | |
396 | 0 | fn statistics(&self) -> Result<Statistics> { |
397 | 0 | Ok(common::compute_record_batch_statistics( |
398 | 0 | &self.data, |
399 | 0 | &self.schema, |
400 | 0 | None, |
401 | 0 | )) |
402 | 0 | } |
403 | | } |
404 | | |
405 | | /// A mock execution plan that errors on a call to execute |
406 | | #[derive(Debug)] |
407 | | pub struct ErrorExec { |
408 | | cache: PlanProperties, |
409 | | } |
410 | | |
411 | | impl Default for ErrorExec { |
412 | 0 | fn default() -> Self { |
413 | 0 | Self::new() |
414 | 0 | } |
415 | | } |
416 | | |
417 | | impl ErrorExec { |
418 | 1 | pub fn new() -> Self { |
419 | 1 | let schema = Arc::new(Schema::new(vec![Field::new( |
420 | 1 | "dummy", |
421 | 1 | DataType::Int64, |
422 | 1 | true, |
423 | 1 | )])); |
424 | 1 | let cache = Self::compute_properties(schema); |
425 | 1 | Self { cache } |
426 | 1 | } |
427 | | |
428 | | /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. |
429 | 1 | fn compute_properties(schema: SchemaRef) -> PlanProperties { |
430 | 1 | let eq_properties = EquivalenceProperties::new(schema); |
431 | 1 | |
432 | 1 | PlanProperties::new( |
433 | 1 | eq_properties, |
434 | 1 | Partitioning::UnknownPartitioning(1), |
435 | 1 | ExecutionMode::Bounded, |
436 | 1 | ) |
437 | 1 | } |
438 | | } |
439 | | |
440 | | impl DisplayAs for ErrorExec { |
441 | 0 | fn fmt_as( |
442 | 0 | &self, |
443 | 0 | t: DisplayFormatType, |
444 | 0 | f: &mut std::fmt::Formatter, |
445 | 0 | ) -> std::fmt::Result { |
446 | 0 | match t { |
447 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
448 | 0 | write!(f, "ErrorExec") |
449 | 0 | } |
450 | 0 | } |
451 | 0 | } |
452 | | } |
453 | | |
454 | | impl ExecutionPlan for ErrorExec { |
455 | 0 | fn name(&self) -> &'static str { |
456 | 0 | Self::static_name() |
457 | 0 | } |
458 | | |
459 | 0 | fn as_any(&self) -> &dyn Any { |
460 | 0 | self |
461 | 0 | } |
462 | | |
463 | 7 | fn properties(&self) -> &PlanProperties { |
464 | 7 | &self.cache |
465 | 7 | } |
466 | | |
467 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
468 | 0 | unimplemented!() |
469 | | } |
470 | | |
471 | 0 | fn with_new_children( |
472 | 0 | self: Arc<Self>, |
473 | 0 | _: Vec<Arc<dyn ExecutionPlan>>, |
474 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
475 | 0 | unimplemented!() |
476 | | } |
477 | | |
478 | | /// Returns a stream which yields data |
479 | 1 | fn execute( |
480 | 1 | &self, |
481 | 1 | partition: usize, |
482 | 1 | _context: Arc<TaskContext>, |
483 | 1 | ) -> Result<SendableRecordBatchStream> { |
484 | 1 | internal_err!("ErrorExec, unsurprisingly, errored in partition {partition}") |
485 | 1 | } |
486 | | } |
487 | | |
488 | | /// A mock execution plan that simply returns the provided statistics |
489 | | #[derive(Debug, Clone)] |
490 | | pub struct StatisticsExec { |
491 | | stats: Statistics, |
492 | | schema: Arc<Schema>, |
493 | | cache: PlanProperties, |
494 | | } |
495 | | impl StatisticsExec { |
496 | 12 | pub fn new(stats: Statistics, schema: Schema) -> Self { |
497 | 12 | assert_eq!( |
498 | 12 | stats |
499 | 12 | .column_statistics.len(), schema.fields().len(), |
500 | 0 | "if defined, the column statistics vector length should be the number of fields" |
501 | | ); |
502 | 12 | let cache = Self::compute_properties(Arc::new(schema.clone())); |
503 | 12 | Self { |
504 | 12 | stats, |
505 | 12 | schema: Arc::new(schema), |
506 | 12 | cache, |
507 | 12 | } |
508 | 12 | } |
509 | | |
510 | | /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. |
511 | 12 | fn compute_properties(schema: SchemaRef) -> PlanProperties { |
512 | 12 | let eq_properties = EquivalenceProperties::new(schema); |
513 | 12 | |
514 | 12 | PlanProperties::new( |
515 | 12 | eq_properties, |
516 | 12 | Partitioning::UnknownPartitioning(2), |
517 | 12 | ExecutionMode::Bounded, |
518 | 12 | ) |
519 | 12 | } |
520 | | } |
521 | | |
522 | | impl DisplayAs for StatisticsExec { |
523 | 0 | fn fmt_as( |
524 | 0 | &self, |
525 | 0 | t: DisplayFormatType, |
526 | 0 | f: &mut std::fmt::Formatter, |
527 | 0 | ) -> std::fmt::Result { |
528 | 0 | match t { |
529 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
530 | 0 | write!( |
531 | 0 | f, |
532 | 0 | "StatisticsExec: col_count={}, row_count={:?}", |
533 | 0 | self.schema.fields().len(), |
534 | 0 | self.stats.num_rows, |
535 | 0 | ) |
536 | 0 | } |
537 | 0 | } |
538 | 0 | } |
539 | | } |
540 | | |
541 | | impl ExecutionPlan for StatisticsExec { |
542 | 0 | fn name(&self) -> &'static str { |
543 | 0 | Self::static_name() |
544 | 0 | } |
545 | | |
546 | 0 | fn as_any(&self) -> &dyn Any { |
547 | 0 | self |
548 | 0 | } |
549 | | |
550 | 111 | fn properties(&self) -> &PlanProperties { |
551 | 111 | &self.cache |
552 | 111 | } |
553 | | |
554 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
555 | 0 | vec![] |
556 | 0 | } |
557 | | |
558 | 0 | fn with_new_children( |
559 | 0 | self: Arc<Self>, |
560 | 0 | _: Vec<Arc<dyn ExecutionPlan>>, |
561 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
562 | 0 | Ok(self) |
563 | 0 | } |
564 | | |
565 | 0 | fn execute( |
566 | 0 | &self, |
567 | 0 | _partition: usize, |
568 | 0 | _context: Arc<TaskContext>, |
569 | 0 | ) -> Result<SendableRecordBatchStream> { |
570 | 0 | unimplemented!("This plan only serves for testing statistics") |
571 | | } |
572 | | |
573 | 28 | fn statistics(&self) -> Result<Statistics> { |
574 | 28 | Ok(self.stats.clone()) |
575 | 28 | } |
576 | | } |
577 | | |
578 | | /// Execution plan that emits streams that block forever. |
579 | | /// |
580 | | /// This is useful to test shutdown / cancelation behavior of certain execution plans. |
581 | | #[derive(Debug)] |
582 | | pub struct BlockingExec { |
583 | | /// Schema that is mocked by this plan. |
584 | | schema: SchemaRef, |
585 | | |
586 | | /// Ref-counting helper to check if the plan and the produced stream are still in memory. |
587 | | refs: Arc<()>, |
588 | | cache: PlanProperties, |
589 | | } |
590 | | |
591 | | impl BlockingExec { |
592 | | /// Create new [`BlockingExec`] with a give schema and number of partitions. |
593 | 11 | pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { |
594 | 11 | let cache = Self::compute_properties(Arc::clone(&schema), n_partitions); |
595 | 11 | Self { |
596 | 11 | schema, |
597 | 11 | refs: Default::default(), |
598 | 11 | cache, |
599 | 11 | } |
600 | 11 | } |
601 | | |
602 | | /// Weak pointer that can be used for ref-counting this execution plan and its streams. |
603 | | /// |
604 | | /// Use [`Weak::strong_count`] to determine if the plan itself and its streams are dropped (should be 0 in that |
605 | | /// case). Note that tokio might take some time to cancel spawned tasks, so you need to wrap this check into a retry |
606 | | /// loop. Use [`assert_strong_count_converges_to_zero`] to archive this. |
607 | 10 | pub fn refs(&self) -> Weak<()> { |
608 | 10 | Arc::downgrade(&self.refs) |
609 | 10 | } |
610 | | |
611 | | /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. |
612 | 11 | fn compute_properties(schema: SchemaRef, n_partitions: usize) -> PlanProperties { |
613 | 11 | let eq_properties = EquivalenceProperties::new(schema); |
614 | 11 | |
615 | 11 | PlanProperties::new( |
616 | 11 | eq_properties, |
617 | 11 | Partitioning::UnknownPartitioning(n_partitions), |
618 | 11 | ExecutionMode::Bounded, |
619 | 11 | ) |
620 | 11 | } |
621 | | } |
622 | | |
623 | | impl DisplayAs for BlockingExec { |
624 | 0 | fn fmt_as( |
625 | 0 | &self, |
626 | 0 | t: DisplayFormatType, |
627 | 0 | f: &mut std::fmt::Formatter, |
628 | 0 | ) -> std::fmt::Result { |
629 | 0 | match t { |
630 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
631 | 0 | write!(f, "BlockingExec",) |
632 | 0 | } |
633 | 0 | } |
634 | 0 | } |
635 | | } |
636 | | |
637 | | impl ExecutionPlan for BlockingExec { |
638 | 0 | fn name(&self) -> &'static str { |
639 | 0 | Self::static_name() |
640 | 0 | } |
641 | | |
642 | 0 | fn as_any(&self) -> &dyn Any { |
643 | 0 | self |
644 | 0 | } |
645 | | |
646 | 58 | fn properties(&self) -> &PlanProperties { |
647 | 58 | &self.cache |
648 | 58 | } |
649 | | |
650 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
651 | 0 | // this is a leaf node and has no children |
652 | 0 | vec![] |
653 | 0 | } |
654 | | |
655 | 0 | fn with_new_children( |
656 | 0 | self: Arc<Self>, |
657 | 0 | _: Vec<Arc<dyn ExecutionPlan>>, |
658 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
659 | 0 | internal_err!("Children cannot be replaced in {self:?}") |
660 | 0 | } |
661 | | |
662 | 7 | fn execute( |
663 | 7 | &self, |
664 | 7 | _partition: usize, |
665 | 7 | _context: Arc<TaskContext>, |
666 | 7 | ) -> Result<SendableRecordBatchStream> { |
667 | 7 | Ok(Box::pin(BlockingStream { |
668 | 7 | schema: Arc::clone(&self.schema), |
669 | 7 | _refs: Arc::clone(&self.refs), |
670 | 7 | })) |
671 | 7 | } |
672 | | } |
673 | | |
674 | | /// A [`RecordBatchStream`] that is pending forever. |
675 | | #[derive(Debug)] |
676 | | pub struct BlockingStream { |
677 | | /// Schema mocked by this stream. |
678 | | schema: SchemaRef, |
679 | | |
680 | | /// Ref-counting helper to check if the stream are still in memory. |
681 | | _refs: Arc<()>, |
682 | | } |
683 | | |
684 | | impl Stream for BlockingStream { |
685 | | type Item = Result<RecordBatch>; |
686 | | |
687 | 6 | fn poll_next( |
688 | 6 | self: Pin<&mut Self>, |
689 | 6 | _cx: &mut Context<'_>, |
690 | 6 | ) -> Poll<Option<Self::Item>> { |
691 | 6 | Poll::Pending |
692 | 6 | } |
693 | | } |
694 | | |
695 | | impl RecordBatchStream for BlockingStream { |
696 | 1 | fn schema(&self) -> SchemaRef { |
697 | 1 | Arc::clone(&self.schema) |
698 | 1 | } |
699 | | } |
700 | | |
701 | | /// Asserts that the strong count of the given [`Weak`] pointer converges to zero. |
702 | | /// |
703 | | /// This might take a while but has a timeout. |
704 | 10 | pub async fn assert_strong_count_converges_to_zero<T>(refs: Weak<T>) { |
705 | 10 | tokio::time::timeout(std::time::Duration::from_secs(10), async { |
706 | | loop { |
707 | 14 | if dbg!(Weak::strong_count(&refs)) == 0 { |
708 | 10 | break; |
709 | 4 | } |
710 | 4 | tokio::time::sleep(std::time::Duration::from_millis(10)).await; |
711 | | } |
712 | 10 | }) |
713 | 4 | .await |
714 | 10 | .unwrap(); |
715 | 10 | } |
716 | | |
717 | | /// |
718 | | |
719 | | /// Execution plan that emits streams that panics. |
720 | | /// |
721 | | /// This is useful to test panic handling of certain execution plans. |
722 | | #[derive(Debug)] |
723 | | pub struct PanicExec { |
724 | | /// Schema that is mocked by this plan. |
725 | | schema: SchemaRef, |
726 | | |
727 | | /// Number of output partitions. Each partition will produce this |
728 | | /// many empty output record batches prior to panicking |
729 | | batches_until_panics: Vec<usize>, |
730 | | cache: PlanProperties, |
731 | | } |
732 | | |
733 | | impl PanicExec { |
734 | | /// Create new [`PanicExec`] with a give schema and number of |
735 | | /// partitions, which will each panic immediately. |
736 | 3 | pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { |
737 | 3 | let batches_until_panics = vec![0; n_partitions]; |
738 | 3 | let cache = Self::compute_properties(Arc::clone(&schema), &batches_until_panics); |
739 | 3 | Self { |
740 | 3 | schema, |
741 | 3 | batches_until_panics, |
742 | 3 | cache, |
743 | 3 | } |
744 | 3 | } |
745 | | |
746 | | /// Set the number of batches prior to panic for a partition |
747 | 2 | pub fn with_partition_panic(mut self, partition: usize, count: usize) -> Self { |
748 | 2 | self.batches_until_panics[partition] = count; |
749 | 2 | self |
750 | 2 | } |
751 | | |
752 | | /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. |
753 | 3 | fn compute_properties( |
754 | 3 | schema: SchemaRef, |
755 | 3 | batches_until_panics: &[usize], |
756 | 3 | ) -> PlanProperties { |
757 | 3 | let eq_properties = EquivalenceProperties::new(schema); |
758 | 3 | let num_partitions = batches_until_panics.len(); |
759 | 3 | |
760 | 3 | PlanProperties::new( |
761 | 3 | eq_properties, |
762 | 3 | Partitioning::UnknownPartitioning(num_partitions), |
763 | 3 | ExecutionMode::Bounded, |
764 | 3 | ) |
765 | 3 | } |
766 | | } |
767 | | |
768 | | impl DisplayAs for PanicExec { |
769 | 0 | fn fmt_as( |
770 | 0 | &self, |
771 | 0 | t: DisplayFormatType, |
772 | 0 | f: &mut std::fmt::Formatter, |
773 | 0 | ) -> std::fmt::Result { |
774 | 0 | match t { |
775 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
776 | 0 | write!(f, "PanicExec",) |
777 | 0 | } |
778 | 0 | } |
779 | 0 | } |
780 | | } |
781 | | |
782 | | impl ExecutionPlan for PanicExec { |
783 | 0 | fn name(&self) -> &'static str { |
784 | 0 | Self::static_name() |
785 | 0 | } |
786 | | |
787 | 0 | fn as_any(&self) -> &dyn Any { |
788 | 0 | self |
789 | 0 | } |
790 | | |
791 | 7 | fn properties(&self) -> &PlanProperties { |
792 | 7 | &self.cache |
793 | 7 | } |
794 | | |
795 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
796 | 0 | // this is a leaf node and has no children |
797 | 0 | vec![] |
798 | 0 | } |
799 | | |
800 | 0 | fn with_new_children( |
801 | 0 | self: Arc<Self>, |
802 | 0 | _: Vec<Arc<dyn ExecutionPlan>>, |
803 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
804 | 0 | internal_err!("Children cannot be replaced in {:?}", self) |
805 | 0 | } |
806 | | |
807 | 14 | fn execute( |
808 | 14 | &self, |
809 | 14 | partition: usize, |
810 | 14 | _context: Arc<TaskContext>, |
811 | 14 | ) -> Result<SendableRecordBatchStream> { |
812 | 14 | Ok(Box::pin(PanicStream { |
813 | 14 | partition, |
814 | 14 | batches_until_panic: self.batches_until_panics[partition], |
815 | 14 | schema: Arc::clone(&self.schema), |
816 | 14 | ready: false, |
817 | 14 | })) |
818 | 14 | } |
819 | | } |
820 | | |
821 | | /// A [`RecordBatchStream`] that yields every other batch and panics |
822 | | /// after `batches_until_panic` batches have been produced. |
823 | | /// |
824 | | /// Useful for testing the behavior of streams on panic |
825 | | #[derive(Debug)] |
826 | | struct PanicStream { |
827 | | /// Which partition was this |
828 | | partition: usize, |
829 | | /// How may batches will be produced until panic |
830 | | batches_until_panic: usize, |
831 | | /// Schema mocked by this stream. |
832 | | schema: SchemaRef, |
833 | | /// Should we return ready ? |
834 | | ready: bool, |
835 | | } |
836 | | |
837 | | impl Stream for PanicStream { |
838 | | type Item = Result<RecordBatch>; |
839 | | |
840 | 27 | fn poll_next( |
841 | 27 | mut self: Pin<&mut Self>, |
842 | 27 | cx: &mut Context<'_>, |
843 | 27 | ) -> Poll<Option<Self::Item>> { |
844 | 27 | if self.batches_until_panic > 0 { |
845 | 14 | if self.ready { |
846 | 7 | self.batches_until_panic -= 1; |
847 | 7 | self.ready = false; |
848 | 7 | let batch = RecordBatch::new_empty(Arc::clone(&self.schema)); |
849 | 7 | return Poll::Ready(Some(Ok(batch))); |
850 | | } else { |
851 | 7 | self.ready = true; |
852 | 7 | // get called again |
853 | 7 | cx.waker().wake_by_ref(); |
854 | 7 | return Poll::Pending; |
855 | | } |
856 | 13 | } |
857 | 13 | panic!("PanickingStream did panic: {}", self.partition) |
858 | 14 | } |
859 | | } |
860 | | |
861 | | impl RecordBatchStream for PanicStream { |
862 | 0 | fn schema(&self) -> SchemaRef { |
863 | 0 | Arc::clone(&self.schema) |
864 | 0 | } |
865 | | } |