-
Notifications
You must be signed in to change notification settings - Fork 42
/
datafusion.rs
274 lines (238 loc) · 7.73 KB
/
datafusion.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
use std::{
any::Any,
collections::Bound,
fmt::{Debug, Formatter},
pin::{pin, Pin},
sync::Arc,
task::{Context, Poll},
};
use arrow::{datatypes::SchemaRef, record_batch::RecordBatch, util::pretty};
use async_stream::stream;
use async_trait::async_trait;
use datafusion::{
catalog::Session,
common::internal_err,
datasource::{TableProvider, TableType},
error::{DataFusionError, Result},
execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext},
physical_expr::EquivalenceProperties,
physical_plan::{
execute_stream, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, PlanProperties,
},
prelude::*,
sql::parser::DFParser,
};
use fusio::path::Path;
use futures_core::Stream;
use futures_util::StreamExt;
use tokio::fs;
use tonbo::{
executor::tokio::TokioExecutor, inmem::immutable::ArrowArrays, record::Record, DbOption, DB,
};
use tonbo_macros::Record;
#[derive(Record, Debug)]
pub struct Music {
#[record(primary_key)]
id: u64,
name: String,
like: i64,
}
struct MusicProvider {
db: Arc<DB<Music, TokioExecutor>>,
}
struct MusicExec {
cache: PlanProperties,
db: Arc<DB<Music, TokioExecutor>>,
projection: Option<Vec<usize>>,
limit: Option<usize>,
range: (Bound<<Music as Record>::Key>, Bound<<Music as Record>::Key>),
}
struct MusicStream {
stream: Pin<Box<dyn Stream<Item = Result<RecordBatch, DataFusionError>> + Send>>,
}
#[async_trait]
impl TableProvider for MusicProvider {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
Music::arrow_schema().clone()
}
fn table_type(&self) -> TableType {
TableType::Base
}
async fn scan(
&self,
_: &dyn Session,
projection: Option<&Vec<usize>>,
_filters: &[Expr],
limit: Option<usize>,
) -> Result<Arc<dyn ExecutionPlan>> {
let mut exec = MusicExec::new(self.db.clone(), projection);
// TODO: filters to range detach
// exec.range =
exec.projection = projection.cloned();
if let Some(projection) = exec.projection.as_mut() {
for index in projection {
*index = index.checked_sub(2).unwrap_or(0);
}
}
exec.limit = limit;
Ok(Arc::new(exec))
}
}
impl MusicExec {
fn new(db: Arc<DB<Music, TokioExecutor>>, projection: Option<&Vec<usize>>) -> Self {
let schema = Music::arrow_schema();
let schema = if let Some(projection) = &projection {
Arc::new(schema.project(projection).unwrap())
} else {
schema.clone()
};
MusicExec {
cache: PlanProperties::new(
EquivalenceProperties::new_with_orderings(schema, &[]),
datafusion::physical_expr::Partitioning::UnknownPartitioning(1),
ExecutionMode::Unbounded,
),
db,
projection: None,
limit: None,
range: (Bound::Unbounded, Bound::Unbounded),
}
}
}
impl Stream for MusicStream {
type Item = Result<RecordBatch>;
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
pin!(&mut self.stream).poll_next(cx)
}
}
impl RecordBatchStream for MusicStream {
fn schema(&self) -> SchemaRef {
Music::arrow_schema().clone()
}
}
impl DisplayAs for MusicExec {
fn fmt_as(&self, _: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result {
let (lower, upper) = self.range;
write!(
f,
"MusicExec: range:({:?}, {:?}), projection: [{:?}], limit: {:?}",
lower, upper, self.projection, self.limit
)
}
}
impl Debug for MusicExec {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.debug_struct("MusicExec")
.field("cache", &self.cache)
.field("limit", &self.limit)
.field("projection", &self.projection)
.field("range", &self.range)
.finish()
}
}
impl ExecutionPlan for MusicExec {
fn name(&self) -> &str {
"MusicExec"
}
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>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
if children.is_empty() {
Ok(self)
} else {
internal_err!("Children cannot be replaced in {self:?}")
}
}
fn execute(&self, _: usize, _: Arc<TaskContext>) -> Result<SendableRecordBatchStream> {
let db = self.db.clone();
let (lower, upper) = self.range;
let limit = self.limit;
let projection = self.projection.clone();
Ok(Box::pin(MusicStream {
stream: Box::pin(stream! {
let txn = db.transaction().await;
let mut scan = txn
.scan((lower.as_ref(), upper.as_ref()));
if let Some(limit) = limit {
scan = scan.limit(limit);
}
if let Some(projection) = projection {
scan = scan.projection(projection.clone());
}
let mut scan = scan.package(8192).await.map_err(|err| DataFusionError::Internal(err.to_string()))?;
while let Some(record) = scan.next().await {
yield Ok(record?.as_record_batch().clone())
}
}),
}))
}
}
#[tokio::main]
async fn main() -> Result<()> {
// make sure the path exists
let _ = fs::create_dir_all("./db_path/music").await;
let options = DbOption::from(Path::from_filesystem_path("./db_path/music").unwrap());
let db = DB::new(options, TokioExecutor::default()).await.unwrap();
for (id, name, like) in [
(0, "welcome".to_string(), 0),
(1, "tonbo".to_string(), 999),
(2, "star".to_string(), 233),
(3, "plz".to_string(), 2),
] {
db.insert(Music { id, name, like }).await.unwrap();
}
let ctx = SessionContext::new();
let provider = MusicProvider { db: Arc::new(db) };
ctx.register_table("music", Arc::new(provider))?;
{
let df = ctx.table("music").await?;
let df = df.select(vec![col("name")])?;
let batches = df.collect().await?;
pretty::print_batches(&batches).unwrap();
}
{
// support sql query for tonbo
let statements = DFParser::parse_sql("select id from music")?;
let plan = ctx
.state()
.statement_to_plan(statements.front().cloned().unwrap())
.await?;
let df = ctx.execute_logical_plan(plan).await?;
let physical_plan = df.create_physical_plan().await?;
let mut stream = execute_stream(physical_plan, ctx.task_ctx())?;
while let Some(maybe_batch) = stream.next().await {
let batch = maybe_batch?;
pretty::print_batches(&[batch]).unwrap();
}
}
// https://github.com/tonbo-io/tonbo/issues/172
{
let statements =
DFParser::parse_sql("select _null, _ts, id, name from music where id = 1")?;
let plan = ctx
.state()
.statement_to_plan(statements.front().cloned().unwrap())
.await?;
let df = ctx.execute_logical_plan(plan).await?;
let physical_plan = df.create_physical_plan().await?;
let mut stream = execute_stream(physical_plan, ctx.task_ctx())?;
while let Some(maybe_batch) = stream.next().await {
let batch = maybe_batch?;
pretty::print_batches(&[batch]).unwrap();
}
}
Ok(())
}