-
Notifications
You must be signed in to change notification settings - Fork 33
/
Copy pathwrite.rs
190 lines (164 loc) · 4.83 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
use crate::mysql::MySQL;
use crate::util::on_conflict::OnConflict;
use crate::util::retriable_error::check_and_mark_retriable_error;
use crate::util::{constraints, to_datafusion_error};
use async_trait::async_trait;
use datafusion::arrow::datatypes::SchemaRef;
use datafusion::{
catalog::Session,
datasource::{TableProvider, TableType},
execution::{SendableRecordBatchStream, TaskContext},
logical_expr::{dml::InsertOp, Expr},
physical_plan::{
insert::{DataSink, DataSinkExec},
metrics::MetricsSet,
DisplayAs, DisplayFormatType, ExecutionPlan,
},
};
use futures::StreamExt;
use mysql_async::TxOpts;
use snafu::ResultExt;
use std::any::Any;
use std::fmt;
use std::sync::Arc;
#[derive(Debug, Clone)]
pub struct MySQLTableWriter {
pub read_provider: Arc<dyn TableProvider>,
mysql: Arc<MySQL>,
on_conflict: Option<OnConflict>,
}
impl MySQLTableWriter {
pub fn create(
read_provider: Arc<dyn TableProvider>,
mysql: MySQL,
on_conflict: Option<OnConflict>,
) -> Arc<Self> {
Arc::new(Self {
read_provider,
mysql: Arc::new(mysql),
on_conflict,
})
}
pub fn mysql(&self) -> Arc<MySQL> {
Arc::clone(&self.mysql)
}
}
#[async_trait]
impl TableProvider for MySQLTableWriter {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.read_provider.schema()
}
fn table_type(&self) -> TableType {
TableType::Base
}
async fn scan(
&self,
state: &dyn Session,
projection: Option<&Vec<usize>>,
filters: &[Expr],
limit: Option<usize>,
) -> datafusion::common::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::common::Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(DataSinkExec::new(
input,
Arc::new(MySQLDataSink::new(
Arc::clone(&self.mysql),
op == InsertOp::Overwrite,
self.on_conflict.clone(),
)),
self.schema(),
None,
)))
}
}
pub struct MySQLDataSink {
pub mysql: Arc<MySQL>,
pub overwrite: bool,
pub on_conflict: Option<OnConflict>,
}
#[async_trait]
impl DataSink for MySQLDataSink {
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 mut num_rows = 0u64;
let mut db_conn = self.mysql.connect().await.map_err(to_datafusion_error)?;
let mysql_conn = MySQL::mysql_conn(&mut db_conn).map_err(to_datafusion_error)?;
let mut conn_guard = mysql_conn.conn.lock().await;
let mut tx = conn_guard
.start_transaction(TxOpts::default())
.await
.context(super::UnableToBeginTransactionSnafu)
.map_err(to_datafusion_error)?;
if self.overwrite {
self.mysql
.delete_all_table_data(&mut tx)
.await
.map_err(to_datafusion_error)?;
}
while let Some(batch) = data.next().await {
let batch = batch.map_err(check_and_mark_retriable_error)?;
let batch_num_rows = batch.num_rows();
if batch_num_rows == 0 {
continue;
}
num_rows += batch_num_rows as u64;
constraints::validate_batch_with_constraints(
&[batch.clone()],
self.mysql.constraints(),
)
.await
.context(super::ConstraintViolationSnafu)
.map_err(to_datafusion_error)?;
self.mysql
.insert_batch(&mut tx, batch, self.on_conflict.clone())
.await
.map_err(to_datafusion_error)?;
}
tx.commit()
.await
.context(super::UnableToCommitMySQLTransactionSnafu)
.map_err(to_datafusion_error)?;
drop(conn_guard);
Ok(num_rows)
}
}
impl MySQLDataSink {
pub fn new(mysql: Arc<MySQL>, overwrite: bool, on_conflict: Option<OnConflict>) -> Self {
Self {
mysql,
overwrite,
on_conflict,
}
}
}
impl fmt::Debug for MySQLDataSink {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "MySQLDataSink")
}
}
impl DisplayAs for MySQLDataSink {
fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "MySQLDataSink")
}
}