-
Notifications
You must be signed in to change notification settings - Fork 33
/
Copy pathwrite.rs
454 lines (388 loc) · 14.9 KB
/
write.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
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
use std::{any::Any, fmt, sync::Arc};
use crate::duckdb::DuckDB;
use crate::sql::db_connection_pool::dbconnection::duckdbconn::DuckDbConnection;
use crate::util::{
constraints,
on_conflict::OnConflict,
retriable_error::{check_and_mark_retriable_error, to_retriable_data_write_error},
};
use arrow::{array::RecordBatch, datatypes::SchemaRef};
use async_trait::async_trait;
use datafusion::catalog::Session;
use datafusion::common::Constraints;
use datafusion::logical_expr::dml::InsertOp;
use datafusion::{
datasource::{TableProvider, TableType},
error::DataFusionError,
execution::{SendableRecordBatchStream, TaskContext},
logical_expr::Expr,
physical_plan::{
insert::{DataSink, DataSinkExec},
metrics::MetricsSet,
DisplayAs, DisplayFormatType, ExecutionPlan,
},
};
use duckdb::{Error as DuckDBError, Transaction};
use futures::StreamExt;
use snafu::prelude::*;
use tokio::sync::mpsc::{self, Receiver, Sender};
use tokio::task::JoinHandle;
use super::to_datafusion_error;
/// A transaction manager that ensures only a single transaction is active at a time
/// for a given DuckDB connection
pub struct DuckDbTransactionManager {
/// The DuckDB connection
conn: DuckDbConnection,
/// The currently active transaction, if any
transaction: Option<Transaction<'static>>,
}
impl<'a> DuckDbTransactionManager {
/// Create a new connection manager with the given connection
pub fn new(conn: DuckDbConnection) -> Self {
Self {
conn,
transaction: None,
}
}
/// Begin a new transaction if one doesn't already exist
pub fn begin(&mut self) -> Result<(), DuckDBError> {
if self.transaction.is_none() {
let tx = self.conn.conn.transaction()?;
// SAFETY: The transaction is tied to the lifetime of the connection - because Rust
// doesn't support self-referential structs, we need to transmute the transaction
// to a static lifetime. We never give out a reference to the transaction with the static lifetime,
// so it's safe to transmute.
self.transaction = Some(unsafe {
std::mem::transmute::<duckdb::Transaction<'_>, duckdb::Transaction<'static>>(tx)
});
}
Ok(())
}
/// Commit the current transaction if one exists and return success/failure
pub fn commit(&mut self) -> Result<(), DuckDBError> {
if let Some(tx) = self.transaction.take() {
tx.commit()?;
}
Ok(())
}
/// Execute a database operation with the current transaction
pub fn tx(&'a self) -> Option<&'a Transaction<'a>> {
self.transaction.as_ref()
}
}
#[derive(Debug, Clone)]
pub struct DuckDBTableWriter {
pub read_provider: Arc<dyn TableProvider>,
duckdb: Arc<DuckDB>,
on_conflict: Option<OnConflict>,
}
impl DuckDBTableWriter {
pub fn create(
read_provider: Arc<dyn TableProvider>,
duckdb: DuckDB,
on_conflict: Option<OnConflict>,
) -> Arc<Self> {
Arc::new(Self {
read_provider,
duckdb: Arc::new(duckdb),
on_conflict,
})
}
pub fn duckdb(&self) -> Arc<DuckDB> {
Arc::clone(&self.duckdb)
}
}
#[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> {
Some(self.duckdb.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>,
overwrite: InsertOp,
) -> datafusion::error::Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(DataSinkExec::new(
input,
Arc::new(DuckDBDataSink::new(
Arc::clone(&self.duckdb),
overwrite,
self.on_conflict.clone(),
)),
self.schema(),
None,
)) as _)
}
}
#[derive(Clone)]
pub(crate) struct DuckDBDataSink {
duckdb: Arc<DuckDB>,
overwrite: InsertOp,
on_conflict: Option<OnConflict>,
}
#[async_trait]
impl DataSink for DuckDBDataSink {
fn as_any(&self) -> &dyn Any {
self
}
fn metrics(&self) -> Option<MetricsSet> {
None
}
async fn write_all(
&self,
mut data: SendableRecordBatchStream,
_context: &Arc<TaskContext>,
) -> datafusion::common::Result<u64> {
let duckdb = Arc::clone(&self.duckdb);
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, mut on_commit_transaction) =
tokio::sync::oneshot::channel();
let duckdb_write_handle: JoinHandle<datafusion::common::Result<u64>> =
tokio::task::spawn_blocking(move || {
let db_conn = Arc::clone(&duckdb)
.connect_sync_direct()
.map_err(to_retriable_data_write_error)?;
let mut tx_manager = DuckDbTransactionManager::new(db_conn);
tx_manager
.begin()
.context(super::UnableToBeginTransactionSnafu)
.map_err(to_datafusion_error)?;
let (num_rows, mut tx_manager) = match **duckdb.constraints() {
[] => try_write_all_no_constraints(duckdb, tx_manager, batch_rx, overwrite)?,
_ => try_write_all_with_constraints(
duckdb,
tx_manager,
batch_rx,
overwrite,
on_conflict,
)?,
};
on_commit_transaction
.try_recv()
.map_err(to_retriable_data_write_error)?;
tx_manager
.commit()
.context(super::UnableToCommitTransactionSnafu)
.map_err(to_retriable_data_write_error)?;
drop(tx_manager);
Ok(num_rows)
});
while let Some(batch) = data.next().await {
let batch = batch.map_err(check_and_mark_retriable_error)?;
constraints::validate_batch_with_constraints(
&[batch.clone()],
self.duckdb.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(
duckdb: Arc<DuckDB>,
overwrite: InsertOp,
on_conflict: Option<OnConflict>,
) -> Self {
Self {
duckdb,
overwrite,
on_conflict,
}
}
}
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")
}
}
/// If there are constraints on the `DuckDB` table, we need to create an empty copy of the target table, write to that table copy and then depending on
/// if the mode is overwrite or not, insert into the target table or drop the target table and rename the current table.
///
/// See: <https://duckdb.org/docs/sql/indexes#over-eager-unique-constraint-checking>
fn try_write_all_with_constraints(
duckdb: Arc<DuckDB>,
mut tx_manager: DuckDbTransactionManager,
mut data_batches: Receiver<RecordBatch>,
overwrite: InsertOp,
on_conflict: Option<OnConflict>,
) -> datafusion::common::Result<(u64, DuckDbTransactionManager)> {
// We want to clone the current table into our insert table
let Some(ref orig_table_creator) = duckdb.table_creator else {
return Err(DataFusionError::Execution(
"Expected table with constraints to have a table creator".to_string(),
));
};
let mut num_rows = 0;
let mut insert_table = orig_table_creator
.create_empty_clone(tx_manager.tx().unwrap())
.map_err(to_datafusion_error)?;
let Some(insert_table_creator) = insert_table.table_creator.take() else {
unreachable!()
};
// Auto-commit after processing this many rows
const MAX_ROWS_PER_COMMIT: usize = 10_000_000;
let mut rows_since_last_commit = 0;
while let Some(batch) = data_batches.blocking_recv() {
num_rows += u64::try_from(batch.num_rows()).map_err(|e| {
DataFusionError::Execution(format!("Unable to convert num_rows() to u64: {e}"))
})?;
rows_since_last_commit += batch.num_rows();
if rows_since_last_commit > MAX_ROWS_PER_COMMIT {
tracing::info!("Committing DuckDB transaction after {rows_since_last_commit} rows.",);
// Commit the current transaction
tx_manager
.commit()
.context(super::UnableToCommitTransactionSnafu)
.map_err(to_datafusion_error)?;
// Create a new transaction
tx_manager
.begin()
.context(super::UnableToBeginTransactionSnafu)
.map_err(to_datafusion_error)?;
rows_since_last_commit = 0;
}
tracing::debug!("Inserting {} rows into cloned table.", batch.num_rows());
insert_table
.insert_batch_no_constraints(tx_manager.tx().unwrap(), &batch)
.map_err(to_datafusion_error)?;
}
if matches!(overwrite, InsertOp::Overwrite) {
insert_table_creator
.replace_table(tx_manager.tx().unwrap(), orig_table_creator)
.map_err(to_datafusion_error)?;
} else {
insert_table
.insert_table_into(tx_manager.tx().unwrap(), &duckdb, on_conflict.as_ref())
.map_err(to_datafusion_error)?;
insert_table_creator
.delete_table(tx_manager.tx().unwrap())
.map_err(to_datafusion_error)?;
}
Ok((num_rows, tx_manager))
}
/// Even if there are no constraints on the `DuckDB` table, we use the temp table approach
/// to be consistent with the constrained case and to help with performance.
fn try_write_all_no_constraints(
duckdb: Arc<DuckDB>,
mut tx_manager: DuckDbTransactionManager,
mut data_batches: Receiver<RecordBatch>,
overwrite: InsertOp,
) -> datafusion::common::Result<(u64, DuckDbTransactionManager)> {
// We want to clone the current table into our insert table
let Some(ref orig_table_creator) = duckdb.table_creator else {
return Err(DataFusionError::Execution(
"Expected table to have a table creator".to_string(),
));
};
let mut num_rows = 0;
let mut insert_table = orig_table_creator
.create_empty_clone(tx_manager.tx().unwrap())
.map_err(to_datafusion_error)?;
let Some(insert_table_creator) = insert_table.table_creator.take() else {
unreachable!()
};
// Auto-commit after processing this many rows
const MAX_ROWS_PER_COMMIT: usize = 10_000_000;
let mut rows_since_last_commit = 0;
while let Some(batch) = data_batches.blocking_recv() {
num_rows += u64::try_from(batch.num_rows()).map_err(|e| {
DataFusionError::Execution(format!("Unable to convert num_rows() to u64: {e}"))
})?;
rows_since_last_commit += batch.num_rows();
if rows_since_last_commit > MAX_ROWS_PER_COMMIT {
tracing::info!("Committing DuckDB transaction after {rows_since_last_commit} rows.",);
// Commit the current transaction
tx_manager
.commit()
.context(super::UnableToCommitTransactionSnafu)
.map_err(to_datafusion_error)?;
// Create a new transaction
tx_manager
.begin()
.context(super::UnableToBeginTransactionSnafu)
.map_err(to_datafusion_error)?;
rows_since_last_commit = 0;
}
tracing::debug!("Inserting {} rows into cloned table.", batch.num_rows());
insert_table
.insert_batch_no_constraints(tx_manager.tx().unwrap(), &batch)
.map_err(to_datafusion_error)?;
}
if matches!(overwrite, InsertOp::Overwrite) {
insert_table_creator
.replace_table(tx_manager.tx().unwrap(), orig_table_creator)
.map_err(to_datafusion_error)?;
} else {
insert_table
.insert_table_into(tx_manager.tx().unwrap(), &duckdb, None)
.map_err(to_datafusion_error)?;
insert_table_creator
.delete_table(tx_manager.tx().unwrap())
.map_err(to_datafusion_error)?;
}
Ok((num_rows, tx_manager))
}