forked from datafusion-contrib/datafusion-table-providers
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathwrite.rs
More file actions
1133 lines (951 loc) · 37.6 KB
/
write.rs
File metadata and controls
1133 lines (951 loc) · 37.6 KB
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
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
use std::time::{SystemTime, UNIX_EPOCH};
use std::{any::Any, fmt, sync::Arc};
use crate::duckdb::DuckDB;
use crate::sql::db_connection_pool::duckdbpool::DuckDbConnectionPool;
use crate::util::{
constraints,
on_conflict::OnConflict,
retriable_error::{check_and_mark_retriable_error, to_retriable_data_write_error},
};
use arrow::array::RecordBatchReader;
use arrow::ffi_stream::FFI_ArrowArrayStream;
use arrow::{array::RecordBatch, datatypes::SchemaRef};
use arrow_schema::ArrowError;
use async_trait::async_trait;
use datafusion::catalog::Session;
use datafusion::common::{Constraints, SchemaExt};
use datafusion::datasource::sink::{DataSink, DataSinkExec};
use datafusion::logical_expr::dml::InsertOp;
use datafusion::{
datasource::{TableProvider, TableType},
error::DataFusionError,
execution::{SendableRecordBatchStream, TaskContext},
logical_expr::Expr,
physical_plan::{metrics::MetricsSet, DisplayAs, DisplayFormatType, ExecutionPlan},
};
use duckdb::Transaction;
use futures::StreamExt;
use snafu::prelude::*;
use tokio::sync::mpsc::{self, Receiver, Sender};
use tokio::task::JoinHandle;
use super::creator::{TableDefinition, TableManager, ViewCreator};
use super::{to_datafusion_error, RelationName};
// checking schemas are equivalent is disabled because it incorrectly marks single-level list fields are different when the name of the field is different
// e.g. List(Field { name: 'a', data_type: Int32 }) != List(Field { name: 'b', data_type: Int32 })
// but, in this case, they are actually equivalent because the field name does not matter for the schema.
// related: https://github.com/apache/arrow-rs/issues/6733#issuecomment-2482582556
const SCHEMA_EQUIVALENCE_ENABLED: bool = false;
#[derive(Default)]
pub struct DuckDBTableWriterBuilder {
read_provider: Option<Arc<dyn TableProvider>>,
pool: Option<Arc<DuckDbConnectionPool>>,
on_conflict: Option<OnConflict>,
table_definition: Option<TableDefinition>,
}
impl DuckDBTableWriterBuilder {
#[must_use]
pub fn new() -> Self {
Self::default()
}
#[must_use]
pub fn with_read_provider(mut self, read_provider: Arc<dyn TableProvider>) -> Self {
self.read_provider = Some(read_provider);
self
}
#[must_use]
pub fn with_pool(mut self, pool: Arc<DuckDbConnectionPool>) -> Self {
self.pool = Some(pool);
self
}
#[must_use]
pub fn set_on_conflict(mut self, on_conflict: Option<OnConflict>) -> Self {
self.on_conflict = on_conflict;
self
}
#[must_use]
pub fn with_table_definition(mut self, table_definition: TableDefinition) -> Self {
self.table_definition = Some(table_definition);
self
}
/// Builds a `DuckDBTableWriter` from the provided configuration.
///
/// # Errors
///
/// Returns an error if any of the required fields are missing:
/// - `read_provider`
/// - `pool`
/// - `table_definition`
pub fn build(self) -> super::Result<DuckDBTableWriter> {
let Some(read_provider) = self.read_provider else {
return Err(super::Error::MissingReadProvider);
};
let Some(pool) = self.pool else {
return Err(super::Error::MissingPool);
};
let Some(table_definition) = self.table_definition else {
return Err(super::Error::MissingTableDefinition);
};
Ok(DuckDBTableWriter {
read_provider,
on_conflict: self.on_conflict,
table_definition: Arc::new(table_definition),
pool,
})
}
}
#[derive(Clone)]
pub struct DuckDBTableWriter {
pub read_provider: Arc<dyn TableProvider>,
pool: Arc<DuckDbConnectionPool>,
table_definition: Arc<TableDefinition>,
on_conflict: Option<OnConflict>,
}
impl std::fmt::Debug for DuckDBTableWriter {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "DuckDBTableWriter")
}
}
impl DuckDBTableWriter {
#[must_use]
pub fn pool(&self) -> Arc<DuckDbConnectionPool> {
Arc::clone(&self.pool)
}
#[must_use]
pub fn table_definition(&self) -> Arc<TableDefinition> {
Arc::clone(&self.table_definition)
}
}
#[async_trait]
impl TableProvider for DuckDBTableWriter {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.read_provider.schema()
}
fn table_type(&self) -> TableType {
TableType::Base
}
fn constraints(&self) -> Option<&Constraints> {
self.table_definition.constraints()
}
async fn scan(
&self,
state: &dyn Session,
projection: Option<&Vec<usize>>,
filters: &[Expr],
limit: Option<usize>,
) -> datafusion::error::Result<Arc<dyn ExecutionPlan>> {
self.read_provider
.scan(state, projection, filters, limit)
.await
}
async fn insert_into(
&self,
_state: &dyn Session,
input: Arc<dyn ExecutionPlan>,
op: InsertOp,
) -> datafusion::error::Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(DataSinkExec::new(
input,
Arc::new(DuckDBDataSink::new(
Arc::clone(&self.pool),
Arc::clone(&self.table_definition),
op,
self.on_conflict.clone(),
self.schema(),
)),
None,
)) as _)
}
}
#[derive(Clone)]
pub(crate) struct DuckDBDataSink {
pool: Arc<DuckDbConnectionPool>,
table_definition: Arc<TableDefinition>,
overwrite: InsertOp,
on_conflict: Option<OnConflict>,
schema: SchemaRef,
}
#[async_trait]
impl DataSink for DuckDBDataSink {
fn as_any(&self) -> &dyn Any {
self
}
fn metrics(&self) -> Option<MetricsSet> {
None
}
fn schema(&self) -> &SchemaRef {
&self.schema
}
async fn write_all(
&self,
mut data: SendableRecordBatchStream,
_context: &Arc<TaskContext>,
) -> datafusion::common::Result<u64> {
let pool = Arc::clone(&self.pool);
let table_definition = Arc::clone(&self.table_definition);
let overwrite = self.overwrite;
let on_conflict = self.on_conflict.clone();
// Limit channel size to a maximum of 100 RecordBatches queued for cases when DuckDB is slower than the writer stream,
// so that we don't significantly increase memory usage. After the maximum RecordBatches are queued, the writer stream will wait
// until DuckDB is able to process more data.
let (batch_tx, batch_rx): (Sender<RecordBatch>, Receiver<RecordBatch>) = mpsc::channel(100);
// Since the main task/stream can be dropped or fail, we use a oneshot channel to signal that all data is received and we should commit the transaction
let (notify_commit_transaction, on_commit_transaction) = tokio::sync::oneshot::channel();
let schema = data.schema();
let duckdb_write_handle: JoinHandle<datafusion::common::Result<u64>> =
tokio::task::spawn_blocking(move || {
let num_rows = match overwrite {
InsertOp::Overwrite => insert_overwrite(
pool,
&table_definition,
batch_rx,
on_conflict.as_ref(),
on_commit_transaction,
schema,
)?,
InsertOp::Append | InsertOp::Replace => insert_append(
pool,
&table_definition,
batch_rx,
on_conflict.as_ref(),
on_commit_transaction,
schema,
)?,
};
Ok(num_rows)
});
while let Some(batch) = data.next().await {
let batch = batch.map_err(check_and_mark_retriable_error)?;
if let Some(constraints) = self.table_definition.constraints() {
constraints::validate_batch_with_constraints(&[batch.clone()], constraints)
.await
.context(super::ConstraintViolationSnafu)
.map_err(to_datafusion_error)?;
}
if let Err(send_error) = batch_tx.send(batch).await {
match duckdb_write_handle.await {
Err(join_error) => {
return Err(DataFusionError::Execution(format!(
"Error writing to DuckDB: {join_error}"
)));
}
Ok(Err(datafusion_error)) => {
return Err(datafusion_error);
}
_ => {
return Err(DataFusionError::Execution(format!(
"Unable to send RecordBatch to DuckDB writer: {send_error}"
)))
}
};
}
}
if notify_commit_transaction.send(()).is_err() {
return Err(DataFusionError::Execution(
"Unable to send message to commit transaction to DuckDB writer.".to_string(),
));
};
// Drop the sender to signal the receiver that no more data is coming
drop(batch_tx);
match duckdb_write_handle.await {
Ok(result) => result,
Err(e) => Err(DataFusionError::Execution(format!(
"Error writing to DuckDB: {e}"
))),
}
}
}
impl DuckDBDataSink {
pub(crate) fn new(
pool: Arc<DuckDbConnectionPool>,
table_definition: Arc<TableDefinition>,
overwrite: InsertOp,
on_conflict: Option<OnConflict>,
schema: SchemaRef,
) -> Self {
Self {
pool,
table_definition,
overwrite,
on_conflict,
schema,
}
}
}
impl std::fmt::Debug for DuckDBDataSink {
fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
write!(f, "DuckDBDataSink")
}
}
impl DisplayAs for DuckDBDataSink {
fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> std::fmt::Result {
write!(f, "DuckDBDataSink")
}
}
fn insert_append(
pool: Arc<DuckDbConnectionPool>,
table_definition: &Arc<TableDefinition>,
batch_rx: Receiver<RecordBatch>,
on_conflict: Option<&OnConflict>,
mut on_commit_transaction: tokio::sync::oneshot::Receiver<()>,
schema: SchemaRef,
) -> datafusion::common::Result<u64> {
let mut db_conn = pool
.connect_sync()
.context(super::DbConnectionPoolSnafu)
.map_err(to_retriable_data_write_error)?;
let duckdb_conn = DuckDB::duckdb_conn(&mut db_conn).map_err(to_retriable_data_write_error)?;
let tx = duckdb_conn
.conn
.transaction()
.context(super::UnableToBeginTransactionSnafu)
.map_err(to_retriable_data_write_error)?;
let append_table = TableManager::new(Arc::clone(table_definition))
.with_internal(false)
.map_err(to_retriable_data_write_error)?;
let should_have_indexes = !append_table.indexes_vec().is_empty();
let has_indexes = !append_table
.current_indexes(&tx)
.map_err(to_retriable_data_write_error)?
.is_empty();
let is_empty_table = append_table
.get_row_count(&tx)
.map_err(to_retriable_data_write_error)?
== 0;
let should_apply_indexes = should_have_indexes && !has_indexes && is_empty_table;
let append_table_schema = append_table
.current_schema(&tx)
.map_err(to_retriable_data_write_error)?;
if SCHEMA_EQUIVALENCE_ENABLED && !schema.equivalent_names_and_types(&append_table_schema) {
return Err(DataFusionError::Execution(
"Schema of the append table does not match the schema of the new append data."
.to_string(),
));
}
tracing::debug!(
"Append load for {table_name}",
table_name = append_table.table_name()
);
let num_rows = write_to_table(&append_table, &tx, schema, batch_rx, on_conflict)
.map_err(to_retriable_data_write_error)?;
on_commit_transaction
.try_recv()
.map_err(to_retriable_data_write_error)?;
tx.commit()
.context(super::UnableToCommitTransactionSnafu)
.map_err(to_retriable_data_write_error)?;
let tx = duckdb_conn
.conn
.transaction()
.context(super::UnableToBeginTransactionSnafu)
.map_err(to_datafusion_error)?;
// apply indexes if new table
if should_apply_indexes {
tracing::debug!(
"Load for table {table_name} complete, applying constraints and indexes.",
table_name = append_table.table_name()
);
append_table
.create_indexes(&tx)
.map_err(to_retriable_data_write_error)?;
}
let primary_keys_match = append_table
.verify_primary_keys_match(&append_table, &tx)
.map_err(to_retriable_data_write_error)?;
let indexes_match = append_table
.verify_indexes_match(&append_table, &tx)
.map_err(to_retriable_data_write_error)?;
if !primary_keys_match {
return Err(DataFusionError::Execution(
"Primary keys do not match between the new table and the existing table.\nEnsure primary key configuration is the same as the existing table, or manually migrate the table.".to_string(),
));
}
if !indexes_match {
return Err(DataFusionError::Execution(
"Indexes do not match between the new table and the existing table.\nEnsure index configuration is the same as the existing table, or manually migrate the table.".to_string(),
));
}
tx.commit()
.context(super::UnableToCommitTransactionSnafu)
.map_err(to_retriable_data_write_error)?;
Ok(num_rows)
}
#[allow(clippy::too_many_lines)]
fn insert_overwrite(
pool: Arc<DuckDbConnectionPool>,
table_definition: &Arc<TableDefinition>,
batch_rx: Receiver<RecordBatch>,
on_conflict: Option<&OnConflict>,
mut on_commit_transaction: tokio::sync::oneshot::Receiver<()>,
schema: SchemaRef,
) -> datafusion::common::Result<u64> {
let cloned_pool = Arc::clone(&pool);
let mut db_conn = pool
.connect_sync()
.context(super::DbConnectionPoolSnafu)
.map_err(to_retriable_data_write_error)?;
let duckdb_conn = DuckDB::duckdb_conn(&mut db_conn).map_err(to_retriable_data_write_error)?;
let tx = duckdb_conn
.conn
.transaction()
.context(super::UnableToBeginTransactionSnafu)
.map_err(to_retriable_data_write_error)?;
let new_table = TableManager::new(Arc::clone(table_definition))
.with_internal(true)
.map_err(to_retriable_data_write_error)?;
new_table
.create_table(cloned_pool, &tx)
.map_err(to_retriable_data_write_error)?;
let existing_tables = new_table
.list_other_internal_tables(&tx)
.map_err(to_retriable_data_write_error)?;
let base_table = new_table
.base_table(&tx)
.map_err(to_retriable_data_write_error)?;
let last_table = match (existing_tables.last(), base_table.as_ref()) {
(Some(internal_table), Some(base_table)) => {
return Err(DataFusionError::Execution(
format!("Failed to insert data for DuckDB - both an internal table and definition base table were found.\nManual table migration is required - delete the table '{internal_table}' or '{base_table}' and try again.",
internal_table = internal_table.0.table_name(),
base_table = base_table.table_name())));
}
(Some((table, _)), None) | (None, Some(table)) => Some(table),
(None, None) => None,
};
if let Some(last_table) = last_table {
let should_have_indexes = !last_table.indexes_vec().is_empty();
let has_indexes = !last_table
.current_indexes(&tx)
.map_err(to_retriable_data_write_error)?
.is_empty();
let is_empty_table = last_table
.get_row_count(&tx)
.map_err(to_retriable_data_write_error)?
== 0;
let should_apply_indexes = should_have_indexes && !has_indexes && is_empty_table;
let last_table_schema = last_table
.current_schema(&tx)
.map_err(to_retriable_data_write_error)?;
let new_table_schema = new_table
.current_schema(&tx)
.map_err(to_retriable_data_write_error)?;
if SCHEMA_EQUIVALENCE_ENABLED
&& !new_table_schema.equivalent_names_and_types(&last_table_schema)
{
return Err(DataFusionError::Execution(
"Schema does not match between the new table and the existing table.".to_string(),
));
}
if !should_apply_indexes {
// compare indexes and primary keys
let primary_keys_match = new_table
.verify_primary_keys_match(last_table, &tx)
.map_err(to_retriable_data_write_error)?;
let indexes_match = new_table
.verify_indexes_match(last_table, &tx)
.map_err(to_retriable_data_write_error)?;
if !primary_keys_match {
return Err(DataFusionError::Execution(
"Primary keys do not match between the new table and the existing table.\nEnsure primary key configuration is the same as the existing table, or manually migrate the table."
.to_string(),
));
}
if !indexes_match {
return Err(DataFusionError::Execution(
"Indexes do not match between the new table and the existing table.\nEnsure index configuration is the same as the existing table, or manually migrate the table.".to_string(),
));
}
}
}
tracing::debug!("Initial load for {}", new_table.table_name());
let num_rows = write_to_table(&new_table, &tx, schema, batch_rx, on_conflict)
.map_err(to_retriable_data_write_error)?;
on_commit_transaction
.try_recv()
.map_err(to_retriable_data_write_error)?;
if let Some(base_table) = base_table {
base_table
.delete_table(&tx)
.map_err(to_retriable_data_write_error)?;
}
new_table
.create_view(&tx)
.map_err(to_retriable_data_write_error)?;
tx.commit()
.context(super::UnableToCommitTransactionSnafu)
.map_err(to_retriable_data_write_error)?;
tracing::debug!(
"Load for table {table_name} complete, applying constraints and indexes.",
table_name = new_table.table_name()
);
let tx = duckdb_conn
.conn
.transaction()
.context(super::UnableToBeginTransactionSnafu)
.map_err(to_datafusion_error)?;
for (table, _) in existing_tables {
table
.delete_table(&tx)
.map_err(to_retriable_data_write_error)?;
}
// Apply constraints and indexes.
new_table
.create_indexes(&tx)
.map_err(to_retriable_data_write_error)?;
tx.commit()
.context(super::UnableToCommitTransactionSnafu)
.map_err(to_retriable_data_write_error)?;
Ok(num_rows)
}
#[allow(clippy::doc_markdown)]
/// Writes a stream of ``RecordBatch``es to a DuckDB table.
fn write_to_table(
table: &TableManager,
tx: &Transaction<'_>,
schema: SchemaRef,
data_batches: Receiver<RecordBatch>,
on_conflict: Option<&OnConflict>,
) -> datafusion::common::Result<u64> {
let stream = FFI_ArrowArrayStream::new(Box::new(RecordBatchReaderFromStream::new(
data_batches,
schema,
)));
let current_ts = SystemTime::now()
.duration_since(UNIX_EPOCH)
.context(super::UnableToGetSystemTimeSnafu)
.map_err(to_datafusion_error)?
.as_millis();
let view_name = format!("__scan_{}_{current_ts}", table.table_name());
tx.register_arrow_scan_view(&view_name, &stream)
.context(super::UnableToRegisterArrowScanViewSnafu)
.map_err(to_datafusion_error)?;
let view = ViewCreator::from_name(RelationName::new(view_name));
let rows = view
.insert_into(table, tx, on_conflict)
.map_err(to_datafusion_error)?;
view.drop(tx).map_err(to_datafusion_error)?;
Ok(rows as u64)
}
struct RecordBatchReaderFromStream {
stream: Receiver<RecordBatch>,
schema: SchemaRef,
}
impl RecordBatchReaderFromStream {
fn new(stream: Receiver<RecordBatch>, schema: SchemaRef) -> Self {
Self { stream, schema }
}
}
impl Iterator for RecordBatchReaderFromStream {
type Item = Result<RecordBatch, ArrowError>;
fn next(&mut self) -> Option<Self::Item> {
self.stream.blocking_recv().map(Ok)
}
}
impl RecordBatchReader for RecordBatchReaderFromStream {
fn schema(&self) -> SchemaRef {
Arc::clone(&self.schema)
}
}
#[cfg(test)]
mod test {
use arrow::array::{Int64Array, StringArray};
use datafusion::physical_plan::memory::MemoryStream;
use super::*;
use crate::{
duckdb::creator::tests::{get_basic_table_definition, get_mem_duckdb, init_tracing},
util::{column_reference::ColumnReference, indexes::IndexType},
};
#[tokio::test]
async fn test_write_to_table_overwrite_without_previous_table() {
// Test scenario: Write to a table with overwrite mode without a previous table
// Expected behavior: Data sink creates a new internal table, writes data to it, and creates a view with the table definition name
let _guard = init_tracing(None);
let pool = get_mem_duckdb();
let table_definition = get_basic_table_definition();
let duckdb_sink = DuckDBDataSink::new(
Arc::clone(&pool),
Arc::clone(&table_definition),
InsertOp::Overwrite,
None,
table_definition.schema(),
);
let data_sink: Arc<dyn DataSink> = Arc::new(duckdb_sink);
// id, name
// 1, "a"
// 2, "b"
let batches = vec![RecordBatch::try_new(
Arc::clone(&table_definition.schema()),
vec![
Arc::new(Int64Array::from(vec![Some(1), Some(2)])),
Arc::new(StringArray::from(vec![Some("a"), Some("b")])),
],
)
.expect("should create a record batch")];
let stream = Box::pin(
MemoryStream::try_new(batches, table_definition.schema(), None).expect("to get stream"),
);
data_sink
.write_all(stream, &Arc::new(TaskContext::default()))
.await
.expect("to write all");
let mut conn = pool.connect_sync().expect("to connect");
let duckdb = DuckDB::duckdb_conn(&mut conn).expect("to get duckdb conn");
let tx = duckdb.conn.transaction().expect("to begin transaction");
let mut internal_tables = table_definition
.list_internal_tables(&tx)
.expect("to list internal tables");
assert_eq!(internal_tables.len(), 1);
let table_name = internal_tables.pop().expect("should have a table").0;
let rows = tx
.query_row(&format!("SELECT COUNT(1) FROM {table_name}"), [], |row| {
row.get::<_, i64>(0)
})
.expect("to get count");
assert_eq!(rows, 2);
// expect a view to be created with the table definition name
let view_rows = tx
.query_row(
&format!(
"SELECT COUNT(1) FROM {view_name}",
view_name = table_definition.name()
),
[],
|row| row.get::<_, i64>(0),
)
.expect("to get count");
assert_eq!(view_rows, 2);
tx.rollback().expect("to rollback");
}
#[tokio::test]
async fn test_write_to_table_overwrite_with_previous_base_table() {
// Test scenario: Write to a table with overwrite mode with a previous base table
// Expected behavior: Data sink creates a new internal table, writes data to it.
// Before creating the view, the base table needs to get dropped as we need to create a view with the same name.
let _guard = init_tracing(None);
let pool = get_mem_duckdb();
let table_definition = get_basic_table_definition();
let cloned_pool = Arc::clone(&pool);
let mut conn = cloned_pool.connect_sync().expect("to connect");
let duckdb = DuckDB::duckdb_conn(&mut conn).expect("to get duckdb conn");
let tx = duckdb.conn.transaction().expect("to begin transaction");
// make an existing table to overwrite
let overwrite_table = TableManager::new(Arc::clone(&table_definition))
.with_internal(false)
.expect("to create table");
overwrite_table
.create_table(Arc::clone(&pool), &tx)
.expect("to create table");
tx.execute(
&format!(
"INSERT INTO {table_name} VALUES (3, 'c')",
table_name = overwrite_table.table_name()
),
[],
)
.expect("to insert");
tx.commit().expect("to commit");
let duckdb_sink = DuckDBDataSink::new(
Arc::clone(&pool),
Arc::clone(&table_definition),
InsertOp::Overwrite,
None,
table_definition.schema(),
);
let data_sink: Arc<dyn DataSink> = Arc::new(duckdb_sink);
// id, name
// 1, "a"
// 2, "b"
let batches = vec![RecordBatch::try_new(
Arc::clone(&table_definition.schema()),
vec![
Arc::new(Int64Array::from(vec![Some(1), Some(2)])),
Arc::new(StringArray::from(vec![Some("a"), Some("b")])),
],
)
.expect("should create a record batch")];
let stream = Box::pin(
MemoryStream::try_new(batches, table_definition.schema(), None).expect("to get stream"),
);
data_sink
.write_all(stream, &Arc::new(TaskContext::default()))
.await
.expect("to write all");
let mut conn = pool.connect_sync().expect("to connect");
let duckdb = DuckDB::duckdb_conn(&mut conn).expect("to get duckdb conn");
let tx = duckdb.conn.transaction().expect("to begin transaction");
let mut internal_tables = table_definition
.list_internal_tables(&tx)
.expect("to list internal tables");
assert_eq!(internal_tables.len(), 1);
let table_name = internal_tables.pop().expect("should have a table").0;
let rows = tx
.query_row(&format!("SELECT COUNT(1) FROM {table_name}"), [], |row| {
row.get::<_, i64>(0)
})
.expect("to get count");
assert_eq!(rows, 2);
let table_creator =
TableManager::from_table_name(Arc::clone(&table_definition), table_name);
let base_table = table_creator.base_table(&tx).expect("to get base table");
assert!(base_table.is_none()); // base table should get deleted
// expect a view to be created with the table definition name
let view_rows = tx
.query_row(
&format!(
"SELECT COUNT(1) FROM {view_name}",
view_name = table_definition.name()
),
[],
|row| row.get::<_, i64>(0),
)
.expect("to get count");
assert_eq!(view_rows, 2);
tx.rollback().expect("to rollback");
}
#[tokio::test]
async fn test_write_to_table_overwrite_with_previous_internal_table() {
// Test scenario: Write to a table with overwrite mode with a previous base table
// Expected behavior: Data sink creates a new internal table, writes data to it.
// Before creating the view, the base table needs to get dropped as we need to create a view with the same name.
let _guard = init_tracing(None);
let pool = get_mem_duckdb();
let table_definition = get_basic_table_definition();
let cloned_pool = Arc::clone(&pool);
let mut conn = cloned_pool.connect_sync().expect("to connect");
let duckdb = DuckDB::duckdb_conn(&mut conn).expect("to get duckdb conn");
let tx = duckdb.conn.transaction().expect("to begin transaction");
// make an existing table to overwrite
let overwrite_table = TableManager::new(Arc::clone(&table_definition))
.with_internal(true)
.expect("to create table");
overwrite_table
.create_table(Arc::clone(&pool), &tx)
.expect("to create table");
tx.execute(
&format!(
"INSERT INTO {table_name} VALUES (3, 'c')",
table_name = overwrite_table.table_name()
),
[],
)
.expect("to insert");
tx.commit().expect("to commit");
let duckdb_sink = DuckDBDataSink::new(
Arc::clone(&pool),
Arc::clone(&table_definition),
InsertOp::Overwrite,
None,
table_definition.schema(),
);
let data_sink: Arc<dyn DataSink> = Arc::new(duckdb_sink);
// id, name
// 1, "a"
// 2, "b"
let batches = vec![RecordBatch::try_new(
Arc::clone(&table_definition.schema()),
vec![
Arc::new(Int64Array::from(vec![Some(1), Some(2)])),
Arc::new(StringArray::from(vec![Some("a"), Some("b")])),
],
)
.expect("should create a record batch")];
let stream = Box::pin(
MemoryStream::try_new(batches, table_definition.schema(), None).expect("to get stream"),
);
data_sink
.write_all(stream, &Arc::new(TaskContext::default()))
.await
.expect("to write all");
let mut conn = pool.connect_sync().expect("to connect");
let duckdb = DuckDB::duckdb_conn(&mut conn).expect("to get duckdb conn");
let tx = duckdb.conn.transaction().expect("to begin transaction");
let mut internal_tables = table_definition
.list_internal_tables(&tx)
.expect("to list internal tables");
assert_eq!(internal_tables.len(), 1);
let table_name = internal_tables.pop().expect("should have a table").0;
let rows = tx
.query_row(&format!("SELECT COUNT(1) FROM {table_name}"), [], |row| {
row.get::<_, i64>(0)
})
.expect("to get count");
assert_eq!(rows, 2);
// expect a view to be created with the table definition name
let view_rows = tx
.query_row(
&format!(
"SELECT COUNT(1) FROM {view_name}",
view_name = table_definition.name()
),
[],
|row| row.get::<_, i64>(0),
)
.expect("to get count");
assert_eq!(view_rows, 2);
tx.rollback().expect("to rollback");
}
#[tokio::test]
async fn test_write_to_table_append_with_previous_table() {
// Test scenario: Write to a table with append mode with a previous table
// Expected behavior: Data sink appends data to the existing table. No new internal table should be created.
// The existing table is re-used.
let _guard = init_tracing(None);
let pool = get_mem_duckdb();
let cloned_pool = Arc::clone(&pool);
let mut conn = cloned_pool.connect_sync().expect("to connect");
let duckdb = DuckDB::duckdb_conn(&mut conn).expect("to get duckdb conn");
let tx = duckdb.conn.transaction().expect("to begin transaction");
let table_definition = get_basic_table_definition();
// make an existing table to append from
let append_table = TableManager::new(Arc::clone(&table_definition))
.with_internal(false)
.expect("to create table");
append_table
.create_table(Arc::clone(&pool), &tx)
.expect("to create table");
tx.execute(
&format!(
"INSERT INTO {table_name} VALUES (3, 'c')",
table_name = append_table.table_name()
),
[],
)
.expect("to insert");
tx.commit().expect("to commit");
let duckdb_sink = DuckDBDataSink::new(
Arc::clone(&pool),
Arc::clone(&table_definition),
InsertOp::Append,
None,
table_definition.schema(),
);
let data_sink: Arc<dyn DataSink> = Arc::new(duckdb_sink);
// id, name
// 1, "a"
// 2, "b"
let batches = vec![RecordBatch::try_new(
Arc::clone(&table_definition.schema()),
vec![
Arc::new(Int64Array::from(vec![Some(1), Some(2)])),
Arc::new(StringArray::from(vec![Some("a"), Some("b")])),
],
)
.expect("should create a record batch")];
let stream = Box::pin(
MemoryStream::try_new(batches, table_definition.schema(), None).expect("to get stream"),
);
data_sink
.write_all(stream, &Arc::new(TaskContext::default()))
.await
.expect("to write all");