forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 5
/
Copy pathmemory.rs
197 lines (177 loc) · 5.56 KB
/
memory.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
// 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.
//! Execution plan for reading in-memory batches of data
use core::fmt;
use std::any::Any;
use std::sync::Arc;
use std::task::{Context, Poll};
use super::{
DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream,
SendableRecordBatchStream,
};
use crate::error::{DataFusionError, Result};
use arrow::datatypes::SchemaRef;
use arrow::error::Result as ArrowResult;
use arrow::record_batch::RecordBatch;
use async_trait::async_trait;
use futures::Stream;
/// Execution plan for reading in-memory batches of data
#[derive(Clone)]
pub struct MemoryExec {
/// The partitions to query
partitions: Vec<Vec<RecordBatch>>,
/// Schema representing the data after the optional projection is applied
schema: SchemaRef,
/// Optional projection
projection: Option<Vec<usize>>,
}
impl fmt::Debug for MemoryExec {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "partitions: [...]")?;
write!(f, "schema: {:?}", self.schema)?;
write!(f, "projection: {:?}", self.projection)
}
}
#[async_trait]
impl ExecutionPlan for MemoryExec {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}
/// Get the schema for this execution plan
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
// this is a leaf node and has no children
vec![]
}
/// Get the output partitioning of this plan
fn output_partitioning(&self) -> Partitioning {
Partitioning::UnknownPartitioning(self.partitions.len())
}
fn with_new_children(
&self,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
if children.is_empty() {
Ok(Arc::new(self.clone()))
} else {
Err(DataFusionError::Internal(format!(
"Children cannot be replaced in {:?}",
self
)))
}
}
async fn execute(&self, partition: usize) -> Result<SendableRecordBatchStream> {
Ok(Box::pin(MemoryStream::try_new(
self.partitions[partition].clone(),
self.schema.clone(),
self.projection.clone(),
)?))
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default => {
let partitions: Vec<_> =
self.partitions.iter().map(|b| b.len()).collect();
write!(
f,
"MemoryExec: partitions={}, partition_sizes={:?}",
partitions.len(),
partitions
)
}
}
}
}
impl MemoryExec {
/// Create a new execution plan for reading in-memory record batches
pub fn try_new(
partitions: &[Vec<RecordBatch>],
schema: SchemaRef,
projection: Option<Vec<usize>>,
) -> Result<Self> {
Ok(Self {
partitions: partitions.to_vec(),
schema,
projection,
})
}
}
/// Iterator over batches
pub(crate) struct MemoryStream {
/// Vector of record batches
data: Vec<RecordBatch>,
/// Schema representing the data
schema: SchemaRef,
/// Optional projection for which columns to load
projection: Option<Vec<usize>>,
/// Index into the data
index: usize,
}
impl MemoryStream {
/// Create an iterator for a vector of record batches
pub fn try_new(
data: Vec<RecordBatch>,
schema: SchemaRef,
projection: Option<Vec<usize>>,
) -> Result<Self> {
Ok(Self {
data,
schema,
projection,
index: 0,
})
}
}
impl Stream for MemoryStream {
type Item = ArrowResult<RecordBatch>;
#[tracing::instrument(level = "trace", skip(self))]
fn poll_next(
mut self: std::pin::Pin<&mut Self>,
_: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
Poll::Ready(if self.index < self.data.len() {
self.index += 1;
let batch = &self.data[self.index - 1];
// apply projection
match &self.projection {
Some(columns) => Some(RecordBatch::try_new(
self.schema.clone(),
columns.iter().map(|i| batch.column(*i).clone()).collect(),
)),
None => Some(Ok(batch.clone())),
}
} else {
None
})
}
fn size_hint(&self) -> (usize, Option<usize>) {
(self.data.len(), Some(self.data.len()))
}
}
impl RecordBatchStream for MemoryStream {
/// Get the schema
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}