datafusion_table_providers/util/test.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
use std::{any::Any, sync::Arc};
use arrow::{array::RecordBatch, datatypes::SchemaRef};
use datafusion::{
common::Statistics,
error::{DataFusionError, Result},
execution::{SendableRecordBatchStream, TaskContext},
physical_expr::EquivalenceProperties,
physical_plan::{
common,
stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter},
DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties,
},
};
/// 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<Result<RecordBatch>>,
schema: SchemaRef,
/// if true (the default), sends data using a separate task to ensure the
/// batches are not available without this stream yielding first
use_task: bool,
cache: PlanProperties,
}
impl MockExec {
/// Create a new `MockExec` with a single partition that returns
/// the specified `Results`s.
///
/// By default, the batches are not produced immediately (the
/// caller has to actually yield and another task must run) to
/// ensure any poll loops are correct. This behavior can be
/// changed with `with_use_task`
pub fn new(data: Vec<Result<RecordBatch>>, schema: SchemaRef) -> Self {
let cache = Self::compute_properties(Arc::clone(&schema));
Self {
data,
schema,
use_task: true,
cache,
}
}
/// If `use_task` is true (the default) then the batches are sent
/// back using a separate task to ensure the underlying stream is
/// not immediately ready
pub fn with_use_task(mut self, use_task: bool) -> Self {
self.use_task = use_task;
self
}
/// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
fn compute_properties(schema: SchemaRef) -> PlanProperties {
let eq_properties = EquivalenceProperties::new(schema);
PlanProperties::new(
eq_properties,
Partitioning::UnknownPartitioning(1),
ExecutionMode::Bounded,
)
}
}
impl DisplayAs for MockExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(f, "MockExec")
}
}
}
}
impl ExecutionPlan for MockExec {
fn name(&self) -> &'static str {
Self::static_name()
}
fn as_any(&self) -> &dyn Any {
self
}
fn properties(&self) -> &PlanProperties {
&self.cache
}
fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
vec![]
}
fn with_new_children(
self: Arc<Self>,
_: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
unimplemented!()
}
/// Returns a stream which yields data
fn execute(
&self,
partition: usize,
_context: Arc<TaskContext>,
) -> 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();
if self.use_task {
let mut builder = RecordBatchReceiverStream::builder(self.schema(), 2);
// send data in order but in a separate task (to ensure
// the batches are not available without the stream
// yielding).
let tx = builder.tx();
builder.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}");
}
}
Ok(())
});
// returned stream simply reads off the rx stream
Ok(builder.build())
} else {
// make an input that will error
let stream = futures::stream::iter(data);
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema(),
stream,
)))
}
}
// Panics if one of the batches is an error
fn statistics(&self) -> Result<Statistics> {
let data: Result<Vec<_>> = self
.data
.iter()
.map(|r| match r {
Ok(batch) => Ok(batch.clone()),
Err(e) => Err(clone_error(e)),
})
.collect();
let data = data?;
Ok(common::compute_record_batch_statistics(
&[data],
&self.schema,
None,
))
}
}
fn clone_error(e: &DataFusionError) -> DataFusionError {
use DataFusionError::*;
match e {
Execution(msg) => Execution(msg.to_string()),
_ => unimplemented!(),
}
}