1use crate::connection::{RemoteDbType, projections_contains};
2use crate::{
3 Connection, ConnectionOptions, DFResult, Literalize, Pool, PoolState, RemoteField,
4 RemoteSchema, RemoteSchemaRef, RemoteSource, RemoteType, SqliteConnectionOptions, SqliteType,
5 literalize_array,
6};
7use datafusion::arrow::array::{
8 ArrayBuilder, ArrayRef, BinaryBuilder, Float64Builder, Int32Builder, Int64Builder, NullBuilder,
9 RecordBatch, RecordBatchOptions, StringBuilder, make_builder,
10};
11use datafusion::arrow::datatypes::{DataType, SchemaRef};
12use datafusion::common::{DataFusionError, project_schema};
13use datafusion::execution::SendableRecordBatchStream;
14use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
15use futures::StreamExt;
16use itertools::Itertools;
17use log::{debug, error};
18use rusqlite::types::ValueRef;
19use rusqlite::{Column, Row, Rows};
20use std::any::Any;
21use std::collections::HashMap;
22use std::path::PathBuf;
23use std::sync::Arc;
24use std::sync::atomic::{AtomicUsize, Ordering};
25
26#[derive(Debug)]
27pub struct SqlitePool {
28 path: PathBuf,
29 connections: Arc<AtomicUsize>,
30}
31
32pub async fn connect_sqlite(options: &SqliteConnectionOptions) -> DFResult<SqlitePool> {
33 let _ = rusqlite::Connection::open(&options.path).map_err(|e| {
34 DataFusionError::Execution(format!("Failed to open sqlite connection: {e:?}"))
35 })?;
36 Ok(SqlitePool {
37 path: options.path.clone(),
38 connections: Arc::new(AtomicUsize::new(0)),
39 })
40}
41
42#[async_trait::async_trait]
43impl Pool for SqlitePool {
44 async fn get(&self) -> DFResult<Arc<dyn Connection>> {
45 self.connections.fetch_add(1, Ordering::SeqCst);
46 Ok(Arc::new(SqliteConnection {
47 path: self.path.clone(),
48 pool_connections: self.connections.clone(),
49 }))
50 }
51
52 async fn state(&self) -> DFResult<PoolState> {
53 Ok(PoolState {
54 connections: self.connections.load(Ordering::SeqCst),
55 idle_connections: 0,
56 })
57 }
58}
59
60#[derive(Debug)]
61pub struct SqliteConnection {
62 path: PathBuf,
63 pool_connections: Arc<AtomicUsize>,
64}
65
66impl Drop for SqliteConnection {
67 fn drop(&mut self) {
68 self.pool_connections.fetch_sub(1, Ordering::SeqCst);
69 }
70}
71
72#[async_trait::async_trait]
73impl Connection for SqliteConnection {
74 fn as_any(&self) -> &dyn Any {
75 self
76 }
77
78 async fn infer_schema(&self, source: &RemoteSource) -> DFResult<RemoteSchemaRef> {
79 let conn = rusqlite::Connection::open(&self.path).map_err(|e| {
80 DataFusionError::Plan(format!("Failed to open sqlite connection: {e:?}"))
81 })?;
82 match source {
83 RemoteSource::Table(table) => {
84 let sql = format!(
86 "PRAGMA table_info({})",
87 RemoteDbType::Sqlite.sql_table_name(table)
88 );
89 let mut stmt = conn.prepare(&sql).map_err(|e| {
90 DataFusionError::Plan(format!("Failed to prepare sqlite statement: {e:?}"))
91 })?;
92 let rows = stmt.query([]).map_err(|e| {
93 DataFusionError::Plan(format!("Failed to query sqlite statement: {e:?}"))
94 })?;
95 let remote_schema = Arc::new(build_remote_schema_for_table(rows)?);
96 Ok(remote_schema)
97 }
98 RemoteSource::Query(_query) => {
99 let sql = RemoteDbType::Sqlite.limit_1_query_if_possible(source);
100 let mut stmt = conn.prepare(&sql).map_err(|e| {
101 DataFusionError::Plan(format!("Failed to prepare sqlite statement: {e:?}"))
102 })?;
103 let columns: Vec<OwnedColumn> =
104 stmt.columns().iter().map(sqlite_col_to_owned_col).collect();
105 let rows = stmt.query([]).map_err(|e| {
106 DataFusionError::Plan(format!("Failed to query sqlite statement: {e:?}"))
107 })?;
108
109 let remote_schema =
110 Arc::new(build_remote_schema_for_query(columns.as_slice(), rows)?);
111 Ok(remote_schema)
112 }
113 }
114 }
115
116 async fn query(
117 &self,
118 conn_options: &ConnectionOptions,
119 source: &RemoteSource,
120 table_schema: SchemaRef,
121 projection: Option<&Vec<usize>>,
122 unparsed_filters: &[String],
123 limit: Option<usize>,
124 ) -> DFResult<SendableRecordBatchStream> {
125 let projected_schema = project_schema(&table_schema, projection)?;
126 let sql = RemoteDbType::Sqlite.rewrite_query(source, unparsed_filters, limit);
127 debug!("[remote-table] executing sqlite query: {sql}");
128
129 let (tx, mut rx) = tokio::sync::mpsc::channel::<DFResult<RecordBatch>>(1);
130 let conn = rusqlite::Connection::open(&self.path).map_err(|e| {
131 DataFusionError::Execution(format!("Failed to open sqlite connection: {e:?}"))
132 })?;
133
134 let projection = projection.cloned();
135 let chunk_size = conn_options.stream_chunk_size();
136
137 spawn_background_task(tx, conn, sql, table_schema, projection, chunk_size);
138
139 let stream = async_stream::stream! {
140 while let Some(batch) = rx.recv().await {
141 yield batch;
142 }
143 };
144 Ok(Box::pin(RecordBatchStreamAdapter::new(
145 projected_schema,
146 stream,
147 )))
148 }
149
150 async fn insert(
151 &self,
152 _conn_options: &ConnectionOptions,
153 literalizer: Arc<dyn Literalize>,
154 table: &[String],
155 remote_schema: RemoteSchemaRef,
156 mut input: SendableRecordBatchStream,
157 ) -> DFResult<usize> {
158 let input_schema = input.schema();
159 let conn = rusqlite::Connection::open(&self.path).map_err(|e| {
160 DataFusionError::Execution(format!("Failed to open sqlite connection: {e:?}"))
161 })?;
162
163 let mut total_count = 0;
164 while let Some(batch) = input.next().await {
165 let batch = batch?;
166
167 let mut columns = Vec::with_capacity(remote_schema.fields.len());
168 for i in 0..batch.num_columns() {
169 let input_field = input_schema.field(i);
170 let remote_field = &remote_schema.fields[i];
171 if remote_field.auto_increment && input_field.is_nullable() {
172 continue;
173 }
174
175 let remote_type = remote_schema.fields[i].remote_type.clone();
176 let array = batch.column(i);
177 let column = literalize_array(literalizer.as_ref(), array, remote_type)?;
178 columns.push(column);
179 }
180
181 let num_rows = columns[0].len();
182 let num_columns = columns.len();
183
184 let mut values = Vec::with_capacity(num_rows);
185 for i in 0..num_rows {
186 let mut value = Vec::with_capacity(num_columns);
187 for col in columns.iter() {
188 value.push(col[i].as_str());
189 }
190 values.push(format!("({})", value.join(",")));
191 }
192
193 let mut col_names = Vec::with_capacity(remote_schema.fields.len());
194 for (remote_field, input_field) in
195 remote_schema.fields.iter().zip(input_schema.fields.iter())
196 {
197 if remote_field.auto_increment && input_field.is_nullable() {
198 continue;
199 }
200 col_names.push(RemoteDbType::Sqlite.sql_identifier(&remote_field.name));
201 }
202
203 let sql = format!(
204 "INSERT INTO {} ({}) VALUES {}",
205 RemoteDbType::Sqlite.sql_table_name(table),
206 col_names.join(","),
207 values.join(",")
208 );
209
210 let count = conn.execute(&sql, []).map_err(|e| {
211 DataFusionError::Execution(format!(
212 "Failed to execute insert statement on sqlite: {e:?}, sql: {sql}"
213 ))
214 })?;
215 total_count += count;
216 }
217
218 Ok(total_count)
219 }
220}
221
222#[derive(Debug)]
223struct OwnedColumn {
224 name: String,
225 decl_type: Option<String>,
226}
227
228fn sqlite_col_to_owned_col(sqlite_col: &Column) -> OwnedColumn {
229 OwnedColumn {
230 name: sqlite_col.name().to_string(),
231 decl_type: sqlite_col.decl_type().map(|x| x.to_string()),
232 }
233}
234
235fn decl_type_to_remote_type(decl_type: &str) -> DFResult<SqliteType> {
236 if [
237 "tinyint", "smallint", "int", "integer", "bigint", "int2", "int4", "int8",
238 ]
239 .contains(&decl_type)
240 {
241 return Ok(SqliteType::Integer);
242 }
243 if ["real", "float", "double", "numeric"].contains(&decl_type) {
244 return Ok(SqliteType::Real);
245 }
246 if decl_type.starts_with("real") || decl_type.starts_with("numeric") {
247 return Ok(SqliteType::Real);
248 }
249 if ["text", "varchar", "char", "string"].contains(&decl_type) {
250 return Ok(SqliteType::Text);
251 }
252 if decl_type.starts_with("char")
253 || decl_type.starts_with("varchar")
254 || decl_type.starts_with("text")
255 {
256 return Ok(SqliteType::Text);
257 }
258 if ["binary", "varbinary", "tinyblob", "blob"].contains(&decl_type) {
259 return Ok(SqliteType::Blob);
260 }
261 if decl_type.starts_with("binary") || decl_type.starts_with("varbinary") {
262 return Ok(SqliteType::Blob);
263 }
264 Err(DataFusionError::NotImplemented(format!(
265 "Unsupported sqlite decl type: {decl_type}",
266 )))
267}
268
269fn build_remote_schema_for_table(mut rows: Rows) -> DFResult<RemoteSchema> {
270 let mut remote_fields = vec![];
271 while let Some(row) = rows.next().map_err(|e| {
272 DataFusionError::Execution(format!("Failed to get next row from sqlite: {e:?}"))
273 })? {
274 let name = row.get::<_, String>(1).map_err(|e| {
275 DataFusionError::Execution(format!("Failed to get col name from sqlite row: {e:?}"))
276 })?;
277 let decl_type = row.get::<_, String>(2).map_err(|e| {
278 DataFusionError::Execution(format!("Failed to get decl type from sqlite row: {e:?}"))
279 })?;
280 let remote_type = decl_type_to_remote_type(&decl_type.to_ascii_lowercase())?;
281 let nullable = row.get::<_, i64>(3).map_err(|e| {
282 DataFusionError::Execution(format!("Failed to get nullable from sqlite row: {e:?}"))
283 })? == 0;
284 remote_fields.push(RemoteField::new(
285 &name,
286 RemoteType::Sqlite(remote_type),
287 nullable,
288 ));
289 }
290 Ok(RemoteSchema::new(remote_fields))
291}
292
293fn build_remote_schema_for_query(
294 columns: &[OwnedColumn],
295 mut rows: Rows,
296) -> DFResult<RemoteSchema> {
297 let mut remote_field_map = HashMap::with_capacity(columns.len());
298 let mut unknown_cols = vec![];
299 for (col_idx, col) in columns.iter().enumerate() {
300 if let Some(decl_type) = &col.decl_type {
301 let remote_type =
302 RemoteType::Sqlite(decl_type_to_remote_type(&decl_type.to_ascii_lowercase())?);
303 remote_field_map.insert(col_idx, RemoteField::new(&col.name, remote_type, true));
304 } else {
305 unknown_cols.push(col_idx);
307 }
308 }
309
310 if !unknown_cols.is_empty() {
311 while let Some(row) = rows.next().map_err(|e| {
312 DataFusionError::Plan(format!("Failed to get next row from sqlite: {e:?}"))
313 })? {
314 let mut to_be_removed = vec![];
315 for col_idx in unknown_cols.iter() {
316 let value_ref = row.get_ref(*col_idx).map_err(|e| {
317 DataFusionError::Plan(format!(
318 "Failed to get value ref for column {col_idx}: {e:?}"
319 ))
320 })?;
321 match value_ref {
322 ValueRef::Null => {}
323 ValueRef::Integer(_) => {
324 remote_field_map.insert(
325 *col_idx,
326 RemoteField::new(
327 columns[*col_idx].name.clone(),
328 RemoteType::Sqlite(SqliteType::Integer),
329 true,
330 ),
331 );
332 to_be_removed.push(*col_idx);
333 }
334 ValueRef::Real(_) => {
335 remote_field_map.insert(
336 *col_idx,
337 RemoteField::new(
338 columns[*col_idx].name.clone(),
339 RemoteType::Sqlite(SqliteType::Real),
340 true,
341 ),
342 );
343 to_be_removed.push(*col_idx);
344 }
345 ValueRef::Text(_) => {
346 remote_field_map.insert(
347 *col_idx,
348 RemoteField::new(
349 columns[*col_idx].name.clone(),
350 RemoteType::Sqlite(SqliteType::Text),
351 true,
352 ),
353 );
354 to_be_removed.push(*col_idx);
355 }
356 ValueRef::Blob(_) => {
357 remote_field_map.insert(
358 *col_idx,
359 RemoteField::new(
360 columns[*col_idx].name.clone(),
361 RemoteType::Sqlite(SqliteType::Blob),
362 true,
363 ),
364 );
365 to_be_removed.push(*col_idx);
366 }
367 }
368 }
369 for col_idx in to_be_removed.iter() {
370 unknown_cols.retain(|&x| x != *col_idx);
371 }
372 if unknown_cols.is_empty() {
373 break;
374 }
375 }
376 }
377
378 if !unknown_cols.is_empty() {
379 return Err(DataFusionError::Plan(format!(
380 "Failed to infer sqlite decl type for columns: {}",
381 unknown_cols
382 .iter()
383 .map(|idx| &columns[*idx].name)
384 .join(", ")
385 )));
386 }
387 let remote_fields = remote_field_map
388 .into_iter()
389 .sorted_by_key(|entry| entry.0)
390 .map(|entry| entry.1)
391 .collect::<Vec<_>>();
392 Ok(RemoteSchema::new(remote_fields))
393}
394
395fn spawn_background_task(
396 tx: tokio::sync::mpsc::Sender<DFResult<RecordBatch>>,
397 conn: rusqlite::Connection,
398 sql: String,
399 table_schema: SchemaRef,
400 projection: Option<Vec<usize>>,
401 chunk_size: usize,
402) {
403 std::thread::spawn(move || {
404 let runtime = match tokio::runtime::Builder::new_current_thread().build() {
405 Ok(runtime) => runtime,
406 Err(e) => {
407 error!("Failed to create tokio runtime to run sqlite query: {e:?}");
408 return;
409 }
410 };
411 let local_set = tokio::task::LocalSet::new();
412 local_set.block_on(&runtime, async move {
413 let mut stmt = match conn.prepare(&sql) {
414 Ok(stmt) => stmt,
415 Err(e) => {
416 let _ = tx
417 .send(Err(DataFusionError::Execution(format!(
418 "Failed to prepare sqlite statement: {e:?}"
419 ))))
420 .await;
421 return;
422 }
423 };
424 let columns: Vec<OwnedColumn> =
425 stmt.columns().iter().map(sqlite_col_to_owned_col).collect();
426 let mut rows = match stmt.query([]) {
427 Ok(rows) => rows,
428 Err(e) => {
429 let _ = tx
430 .send(Err(DataFusionError::Execution(format!(
431 "Failed to query sqlite statement: {e:?}"
432 ))))
433 .await;
434 return;
435 }
436 };
437
438 loop {
439 let (batch, is_empty) = match rows_to_batch(
440 &mut rows,
441 &table_schema,
442 &columns,
443 projection.as_ref(),
444 chunk_size,
445 ) {
446 Ok((batch, is_empty)) => (batch, is_empty),
447 Err(e) => {
448 let _ = tx
449 .send(Err(DataFusionError::Execution(format!(
450 "Failed to convert rows to batch: {e:?}"
451 ))))
452 .await;
453 return;
454 }
455 };
456 if is_empty {
457 break;
458 }
459 if tx.send(Ok(batch)).await.is_err() {
460 return;
461 }
462 }
463 });
464 });
465}
466
467fn rows_to_batch(
468 rows: &mut Rows,
469 table_schema: &SchemaRef,
470 columns: &[OwnedColumn],
471 projection: Option<&Vec<usize>>,
472 chunk_size: usize,
473) -> DFResult<(RecordBatch, bool)> {
474 let projected_schema = project_schema(table_schema, projection)?;
475 let mut array_builders = vec![];
476 for field in table_schema.fields() {
477 let builder = make_builder(field.data_type(), 1000);
478 array_builders.push(builder);
479 }
480
481 let mut is_empty = true;
482 let mut row_count = 0;
483 while let Some(row) = rows.next().map_err(|e| {
484 DataFusionError::Execution(format!("Failed to get next row from sqlite: {e:?}"))
485 })? {
486 is_empty = false;
487 row_count += 1;
488 append_rows_to_array_builders(
489 row,
490 table_schema,
491 columns,
492 projection,
493 array_builders.as_mut_slice(),
494 )?;
495 if row_count >= chunk_size {
496 break;
497 }
498 }
499
500 let projected_columns = array_builders
501 .into_iter()
502 .enumerate()
503 .filter(|(idx, _)| projections_contains(projection, *idx))
504 .map(|(_, mut builder)| builder.finish())
505 .collect::<Vec<ArrayRef>>();
506 let options = RecordBatchOptions::new().with_row_count(Some(row_count));
507 Ok((
508 RecordBatch::try_new_with_options(projected_schema, projected_columns, &options)?,
509 is_empty,
510 ))
511}
512
513macro_rules! handle_primitive_type {
514 ($builder:expr, $field:expr, $col:expr, $builder_ty:ty, $value_ty:ty, $row:expr, $index:expr) => {{
515 let builder = $builder
516 .as_any_mut()
517 .downcast_mut::<$builder_ty>()
518 .unwrap_or_else(|| {
519 panic!(
520 "Failed to downcast builder to {} for {:?} and {:?}",
521 stringify!($builder_ty),
522 $field,
523 $col
524 )
525 });
526
527 let v: Option<$value_ty> = $row.get($index).map_err(|e| {
528 DataFusionError::Execution(format!(
529 "Failed to get optional {} value for {:?} and {:?}: {e:?}",
530 stringify!($value_ty),
531 $field,
532 $col
533 ))
534 })?;
535
536 match v {
537 Some(v) => builder.append_value(v),
538 None => builder.append_null(),
539 }
540 }};
541}
542
543fn append_rows_to_array_builders(
544 row: &Row,
545 table_schema: &SchemaRef,
546 columns: &[OwnedColumn],
547 projection: Option<&Vec<usize>>,
548 array_builders: &mut [Box<dyn ArrayBuilder>],
549) -> DFResult<()> {
550 for (idx, field) in table_schema.fields.iter().enumerate() {
551 if !projections_contains(projection, idx) {
552 continue;
553 }
554 let builder = &mut array_builders[idx];
555 let col = columns.get(idx);
556 match field.data_type() {
557 DataType::Null => {
558 let builder = builder
559 .as_any_mut()
560 .downcast_mut::<NullBuilder>()
561 .expect("Failed to downcast builder to NullBuilder");
562 builder.append_null();
563 }
564 DataType::Int32 => {
565 handle_primitive_type!(builder, field, col, Int32Builder, i32, row, idx);
566 }
567 DataType::Int64 => {
568 handle_primitive_type!(builder, field, col, Int64Builder, i64, row, idx);
569 }
570 DataType::Float64 => {
571 handle_primitive_type!(builder, field, col, Float64Builder, f64, row, idx);
572 }
573 DataType::Utf8 => {
574 handle_primitive_type!(builder, field, col, StringBuilder, String, row, idx);
575 }
576 DataType::Binary => {
577 handle_primitive_type!(builder, field, col, BinaryBuilder, Vec<u8>, row, idx);
578 }
579 _ => {
580 return Err(DataFusionError::NotImplemented(format!(
581 "Unsupported data type {} for col: {:?}",
582 field.data_type(),
583 col
584 )));
585 }
586 }
587 }
588 Ok(())
589}