-
Notifications
You must be signed in to change notification settings - Fork 33
/
Copy pathsql_table.rs
184 lines (157 loc) · 4.91 KB
/
sql_table.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
use crate::sql::db_connection_pool::DbConnectionPool;
use crate::sql::sql_provider_datafusion::Engine;
use async_trait::async_trait;
use datafusion::catalog::Session;
use futures::TryStreamExt;
use std::fmt::Display;
use std::{any::Any, fmt, sync::Arc};
use crate::sql::sql_provider_datafusion::{
get_stream, to_execution_error, Result as SqlResult, SqlExec, SqlTable,
};
use datafusion::{
arrow::datatypes::SchemaRef,
datasource::TableProvider,
error::Result as DataFusionResult,
execution::TaskContext,
logical_expr::{Expr, TableProviderFilterPushDown, TableType},
physical_plan::{
stream::RecordBatchStreamAdapter, DisplayAs, DisplayFormatType, ExecutionPlan,
PlanProperties, SendableRecordBatchStream,
},
sql::TableReference,
};
pub struct SQLiteTable<T: 'static, P: 'static> {
pub(crate) base_table: SqlTable<T, P>,
}
impl<T, P> fmt::Debug for SQLiteTable<T, P> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "SQLiteTable {}", self.base_table.name())
}
}
impl<T, P> SQLiteTable<T, P> {
pub fn new_with_schema(
pool: &Arc<dyn DbConnectionPool<T, P> + Send + Sync>,
schema: impl Into<SchemaRef>,
table_reference: impl Into<TableReference>,
) -> Self {
let base_table = SqlTable::new_with_schema(
"sqlite",
pool,
schema,
table_reference,
Some(Engine::SQLite),
);
Self { base_table }
}
fn create_physical_plan(
&self,
projection: Option<&Vec<usize>>,
schema: &SchemaRef,
sql: String,
) -> DataFusionResult<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(SQLiteSqlExec::new(
projection,
schema,
self.base_table.clone_pool(),
sql,
)?))
}
}
#[async_trait]
impl<T, P> TableProvider for SQLiteTable<T, P> {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.base_table.schema()
}
fn table_type(&self) -> TableType {
self.base_table.table_type()
}
fn supports_filters_pushdown(
&self,
filters: &[&Expr],
) -> DataFusionResult<Vec<TableProviderFilterPushDown>> {
self.base_table.supports_filters_pushdown(filters)
}
async fn scan(
&self,
_state: &dyn Session,
projection: Option<&Vec<usize>>,
filters: &[Expr],
limit: Option<usize>,
) -> DataFusionResult<Arc<dyn ExecutionPlan>> {
let sql = self.base_table.scan_to_sql(projection, filters, limit)?;
return self.create_physical_plan(projection, &self.schema(), sql);
}
}
impl<T, P> Display for SQLiteTable<T, P> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "SQLiteTable {}", self.base_table.name())
}
}
#[derive(Clone)]
struct SQLiteSqlExec<T, P> {
base_exec: SqlExec<T, P>,
}
impl<T, P> SQLiteSqlExec<T, P> {
fn new(
projection: Option<&Vec<usize>>,
schema: &SchemaRef,
pool: Arc<dyn DbConnectionPool<T, P> + Send + Sync>,
sql: String,
) -> DataFusionResult<Self> {
let base_exec = SqlExec::new(projection, schema, pool, sql)?;
Ok(Self { base_exec })
}
fn sql(&self) -> SqlResult<String> {
self.base_exec.sql()
}
}
impl<T, P> std::fmt::Debug for SQLiteSqlExec<T, P> {
fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
let sql = self.sql().unwrap_or_default();
write!(f, "SQLiteSqlExec sql={sql}")
}
}
impl<T, P> DisplayAs for SQLiteSqlExec<T, P> {
fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> std::fmt::Result {
let sql = self.sql().unwrap_or_default();
write!(f, "SQLiteSqlExec sql={sql}")
}
}
impl<T: 'static, P: 'static> ExecutionPlan for SQLiteSqlExec<T, P> {
fn name(&self) -> &'static str {
"SQLiteSqlExec"
}
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.base_exec.schema()
}
fn properties(&self) -> &PlanProperties {
self.base_exec.properties()
}
fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
self.base_exec.children()
}
fn with_new_children(
self: Arc<Self>,
_children: Vec<Arc<dyn ExecutionPlan>>,
) -> DataFusionResult<Arc<dyn ExecutionPlan>> {
Ok(self)
}
fn execute(
&self,
_partition: usize,
_context: Arc<TaskContext>,
) -> DataFusionResult<SendableRecordBatchStream> {
let sql = self.sql().map_err(to_execution_error)?;
tracing::debug!("SQLiteSqlExec sql: {sql}");
let fut = get_stream(self.base_exec.clone_pool(), sql, Arc::clone(&self.schema()));
let stream = futures::stream::once(fut).try_flatten();
let schema = Arc::clone(&self.schema());
Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream)))
}
}