forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathexec.rs
371 lines (320 loc) · 9.78 KB
/
exec.rs
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
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! Simple iterator over batches for use in testing
use async_trait::async_trait;
use std::{
any::Any,
sync::Arc,
task::{Context, Poll},
};
use tokio::sync::Barrier;
use arrow::{
datatypes::{DataType, Field, Schema, SchemaRef},
error::{ArrowError, Result as ArrowResult},
record_batch::RecordBatch,
};
use futures::Stream;
use crate::physical_plan::{
DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream,
SendableRecordBatchStream,
};
use crate::{
error::{DataFusionError, Result},
physical_plan::stream::RecordBatchReceiverStream,
};
/// Index into the data that has been returned so far
#[derive(Debug, Default, Clone)]
pub struct BatchIndex {
inner: std::sync::Arc<std::sync::Mutex<usize>>,
}
impl BatchIndex {
/// Return the current index
pub fn value(&self) -> usize {
let inner = self.inner.lock().unwrap();
*inner
}
// increment the current index by one
pub fn incr(&self) {
let mut inner = self.inner.lock().unwrap();
*inner += 1;
}
}
/// Iterator over batches
#[derive(Debug, Default)]
pub(crate) struct TestStream {
/// Vector of record batches
data: Vec<RecordBatch>,
/// Index into the data that has been returned so far
index: BatchIndex,
}
impl TestStream {
/// Create an iterator for a vector of record batches. Assumes at
/// least one entry in data (for the schema)
pub fn new(data: Vec<RecordBatch>) -> Self {
Self {
data,
..Default::default()
}
}
/// Return a handle to the index counter for this stream
pub fn index(&self) -> BatchIndex {
self.index.clone()
}
}
impl Stream for TestStream {
type Item = ArrowResult<RecordBatch>;
fn poll_next(
self: std::pin::Pin<&mut Self>,
_: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
let next_batch = self.index.value();
Poll::Ready(if next_batch < self.data.len() {
let next_batch = self.index.value();
self.index.incr();
Some(Ok(self.data[next_batch].clone()))
} else {
None
})
}
fn size_hint(&self) -> (usize, Option<usize>) {
(self.data.len(), Some(self.data.len()))
}
}
impl RecordBatchStream for TestStream {
/// Get the schema
fn schema(&self) -> SchemaRef {
self.data[0].schema()
}
}
/// A Mock ExecutionPlan that can be used for writing tests of other ExecutionPlans
///
#[derive(Debug)]
pub struct MockExec {
/// the results to send back
data: Vec<ArrowResult<RecordBatch>>,
schema: SchemaRef,
}
impl MockExec {
/// Create a new exec with a single partition that returns the
/// record batches in this Exec. Note the batches are not produced
/// immediately (the caller has to actually yield and another task
/// must run) to ensure any poll loops are correct.
pub fn new(data: Vec<ArrowResult<RecordBatch>>, schema: SchemaRef) -> Self {
Self { data, schema }
}
}
#[async_trait]
impl ExecutionPlan for MockExec {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
Arc::clone(&self.schema)
}
fn output_partitioning(&self) -> Partitioning {
Partitioning::UnknownPartitioning(1)
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
unimplemented!()
}
fn with_new_children(
&self,
_children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
unimplemented!()
}
/// Returns a stream which yields data
async fn execute(&self, partition: usize) -> Result<SendableRecordBatchStream> {
assert_eq!(partition, 0);
// Result doesn't implement clone, so do it ourself
let data: Vec<_> = self
.data
.iter()
.map(|r| match r {
Ok(batch) => Ok(batch.clone()),
Err(e) => Err(clone_error(e)),
})
.collect();
let (tx, rx) = tokio::sync::mpsc::channel(2);
// task simply sends data in order but in a separate
// thread (to ensure the batches are not available without the
// DelayedStream yielding).
tokio::task::spawn(async move {
for batch in data {
println!("Sending batch via delayed stream");
if let Err(e) = tx.send(batch).await {
println!("ERROR batch via delayed stream: {}", e);
}
}
});
// returned stream simply reads off the rx stream
Ok(RecordBatchReceiverStream::create(&self.schema, rx))
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default => {
write!(f, "MockExec")
}
}
}
}
fn clone_error(e: &ArrowError) -> ArrowError {
use ArrowError::*;
match e {
ComputeError(msg) => ComputeError(msg.to_string()),
_ => unimplemented!(),
}
}
/// A Mock ExecutionPlan that does not start producing input until a
/// barrier is called
///
#[derive(Debug)]
pub struct BarrierExec {
/// partitions to send back
data: Vec<Vec<RecordBatch>>,
schema: SchemaRef,
/// all streams wait on this barrier to produce
barrier: Arc<Barrier>,
}
impl BarrierExec {
/// Create a new exec with some number of partitions.
pub fn new(data: Vec<Vec<RecordBatch>>, schema: SchemaRef) -> Self {
// wait for all streams and the input
let barrier = Arc::new(Barrier::new(data.len() + 1));
Self {
data,
schema,
barrier,
}
}
/// wait until all the input streams and this function is ready
pub async fn wait(&self) {
println!("BarrierExec::wait waiting on barrier");
self.barrier.wait().await;
println!("BarrierExec::wait done waiting");
}
}
#[async_trait]
impl ExecutionPlan for BarrierExec {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
Arc::clone(&self.schema)
}
fn output_partitioning(&self) -> Partitioning {
Partitioning::UnknownPartitioning(self.data.len())
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
unimplemented!()
}
fn with_new_children(
&self,
_children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
unimplemented!()
}
/// Returns a stream which yields data
async fn execute(&self, partition: usize) -> Result<SendableRecordBatchStream> {
assert!(partition < self.data.len());
let (tx, rx) = tokio::sync::mpsc::channel(2);
// task simply sends data in order after barrier is reached
let data = self.data[partition].clone();
let b = self.barrier.clone();
tokio::task::spawn(async move {
println!("Partition {} waiting on barrier", partition);
b.wait().await;
for batch in data {
println!("Partition {} sending batch", partition);
if let Err(e) = tx.send(Ok(batch)).await {
println!("ERROR batch via barrier stream stream: {}", e);
}
}
});
// returned stream simply reads off the rx stream
Ok(RecordBatchReceiverStream::create(&self.schema, rx))
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default => {
write!(f, "BarrierExec")
}
}
}
}
/// A mock execution plan that errors on a call to execute
#[derive(Debug)]
pub struct ErrorExec {
schema: SchemaRef,
}
impl ErrorExec {
pub fn new() -> Self {
let schema = Arc::new(Schema::new(vec![Field::new(
"dummy",
DataType::Int64,
true,
)]));
Self { schema }
}
}
#[async_trait]
impl ExecutionPlan for ErrorExec {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
Arc::clone(&self.schema)
}
fn output_partitioning(&self) -> Partitioning {
Partitioning::UnknownPartitioning(1)
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
unimplemented!()
}
fn with_new_children(
&self,
_children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
unimplemented!()
}
/// Returns a stream which yields data
async fn execute(&self, partition: usize) -> Result<SendableRecordBatchStream> {
Err(DataFusionError::Internal(format!(
"ErrorExec, unsurprisingly, errored in partition {}",
partition
)))
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default => {
write!(f, "ErrorExec")
}
}
}
}