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
255 lines (219 loc) · 6.89 KB
/
write.rs
File metadata and controls
255 lines (219 loc) · 6.89 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
use std::{any::Any, fmt, sync::Arc};
use arrow::datatypes::SchemaRef;
use arrow_schema::{DataType, Field, Schema};
use async_trait::async_trait;
use datafusion::{
catalog::Session,
common::{Constraints, SchemaExt},
datasource::{
sink::{DataSink, DataSinkExec},
TableProvider, TableType,
},
execution::{SendableRecordBatchStream, TaskContext},
logical_expr::{dml::InsertOp, Expr},
physical_plan::{metrics::MetricsSet, DisplayAs, DisplayFormatType, ExecutionPlan},
};
use futures::StreamExt;
use snafu::prelude::*;
use crate::util::{
constraints, on_conflict::OnConflict, retriable_error::check_and_mark_retriable_error,
};
use crate::postgres::Postgres;
use super::to_datafusion_error;
#[derive(Debug, Clone)]
pub struct PostgresTableWriter {
pub read_provider: Arc<dyn TableProvider>,
postgres: Arc<Postgres>,
on_conflict: Option<OnConflict>,
}
impl PostgresTableWriter {
pub fn create(
read_provider: Arc<dyn TableProvider>,
postgres: Postgres,
on_conflict: Option<OnConflict>,
) -> Arc<Self> {
Arc::new(Self {
read_provider,
postgres: Arc::new(postgres),
on_conflict,
})
}
pub fn postgres(&self) -> Arc<Postgres> {
Arc::clone(&self.postgres)
}
}
#[async_trait]
impl TableProvider for PostgresTableWriter {
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.postgres.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(PostgresDataSink::new(
Arc::clone(&self.postgres),
op,
self.on_conflict.clone(),
self.schema(),
)),
None,
)) as _)
}
}
#[derive(Clone)]
struct PostgresDataSink {
postgres: Arc<Postgres>,
overwrite: InsertOp,
on_conflict: Option<OnConflict>,
schema: SchemaRef,
}
#[async_trait]
impl DataSink for PostgresDataSink {
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 mut num_rows = 0;
let mut db_conn = self.postgres.connect().await.map_err(to_datafusion_error)?;
let postgres_conn = Postgres::postgres_conn(&mut db_conn).map_err(to_datafusion_error)?;
let tx = postgres_conn
.conn
.transaction()
.await
.context(super::UnableToBeginTransactionSnafu)
.map_err(to_datafusion_error)?;
if matches!(self.overwrite, InsertOp::Overwrite) {
self.postgres
.delete_all_table_data(&tx)
.await
.map_err(to_datafusion_error)?;
}
let postgres_fields = self
.postgres
.schema
.fields
.iter()
.map(|f| {
Arc::new(Field::new(
f.name(),
if f.data_type() == &DataType::LargeUtf8 {
DataType::Utf8
} else {
f.data_type().clone()
},
f.is_nullable(),
))
})
.collect::<Vec<_>>();
let postgres_schema = Arc::new(Schema::new(postgres_fields));
while let Some(batch) = data.next().await {
let batch = batch.map_err(check_and_mark_retriable_error)?;
// for the purposes of PostgreSQL, LargeUtf8 is equivalent to Utf8
// because Postgres physically cannot store anything larger than 1Gb in text (VARCHAR)
// normalize LargeUtf8 fields to Utf8 for both the incoming batch, and Postgres if it happens to specify any
let batch_fields = batch
.schema_ref()
.fields()
.iter()
.map(|f| {
Arc::new(Field::new(
f.name(),
if f.data_type() == &DataType::LargeUtf8 {
DataType::Utf8
} else {
f.data_type().clone()
},
f.is_nullable(),
))
})
.collect::<Vec<_>>();
let batch_schema = Arc::new(Schema::new(batch_fields));
if !Arc::clone(&postgres_schema).equivalent_names_and_types(&batch_schema) {
return Err(to_datafusion_error(super::Error::SchemaValidationError {
table_name: self.postgres.table.to_string(),
}));
}
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.postgres.constraints(),
)
.await
.context(super::ConstraintViolationSnafu)
.map_err(to_datafusion_error)?;
self.postgres
.insert_batch(&tx, batch, self.on_conflict.clone())
.await
.map_err(to_datafusion_error)?;
}
tx.commit()
.await
.context(super::UnableToCommitPostgresTransactionSnafu)
.map_err(to_datafusion_error)?;
Ok(num_rows)
}
}
impl PostgresDataSink {
fn new(
postgres: Arc<Postgres>,
overwrite: InsertOp,
on_conflict: Option<OnConflict>,
schema: SchemaRef,
) -> Self {
Self {
postgres,
overwrite,
on_conflict,
schema,
}
}
}
impl std::fmt::Debug for PostgresDataSink {
fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
write!(f, "PostgresDataSink")
}
}
impl DisplayAs for PostgresDataSink {
fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> std::fmt::Result {
write!(f, "PostgresDataSink")
}
}