/Users/andrewlamb/Software/datafusion/datafusion/physical-plan/src/insert.rs
Line | Count | Source (jump to first uncovered line) |
1 | | // Licensed to the Apache Software Foundation (ASF) under one |
2 | | // or more contributor license agreements. See the NOTICE file |
3 | | // distributed with this work for additional information |
4 | | // regarding copyright ownership. The ASF licenses this file |
5 | | // to you under the Apache License, Version 2.0 (the |
6 | | // "License"); you may not use this file except in compliance |
7 | | // with the License. You may obtain a copy of the License at |
8 | | // |
9 | | // http://www.apache.org/licenses/LICENSE-2.0 |
10 | | // |
11 | | // Unless required by applicable law or agreed to in writing, |
12 | | // software distributed under the License is distributed on an |
13 | | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
14 | | // KIND, either express or implied. See the License for the |
15 | | // specific language governing permissions and limitations |
16 | | // under the License. |
17 | | |
18 | | //! Execution plan for writing data to [`DataSink`]s |
19 | | |
20 | | use std::any::Any; |
21 | | use std::fmt; |
22 | | use std::fmt::Debug; |
23 | | use std::sync::Arc; |
24 | | |
25 | | use super::{ |
26 | | execute_input_stream, DisplayAs, DisplayFormatType, ExecutionPlan, |
27 | | ExecutionPlanProperties, Partitioning, PlanProperties, SendableRecordBatchStream, |
28 | | }; |
29 | | use crate::metrics::MetricsSet; |
30 | | use crate::stream::RecordBatchStreamAdapter; |
31 | | |
32 | | use arrow::datatypes::SchemaRef; |
33 | | use arrow::record_batch::RecordBatch; |
34 | | use arrow_array::{ArrayRef, UInt64Array}; |
35 | | use arrow_schema::{DataType, Field, Schema}; |
36 | | use datafusion_common::{internal_err, Result}; |
37 | | use datafusion_execution::TaskContext; |
38 | | use datafusion_physical_expr::{Distribution, EquivalenceProperties}; |
39 | | |
40 | | use async_trait::async_trait; |
41 | | use datafusion_physical_expr_common::sort_expr::LexRequirement; |
42 | | use futures::StreamExt; |
43 | | |
44 | | /// `DataSink` implements writing streams of [`RecordBatch`]es to |
45 | | /// user defined destinations. |
46 | | /// |
47 | | /// The `Display` impl is used to format the sink for explain plan |
48 | | /// output. |
49 | | #[async_trait] |
50 | | pub trait DataSink: DisplayAs + Debug + Send + Sync { |
51 | | /// Returns the data sink as [`Any`](std::any::Any) so that it can be |
52 | | /// downcast to a specific implementation. |
53 | | fn as_any(&self) -> &dyn Any; |
54 | | |
55 | | /// Return a snapshot of the [MetricsSet] for this |
56 | | /// [DataSink]. |
57 | | /// |
58 | | /// See [ExecutionPlan::metrics()] for more details |
59 | | fn metrics(&self) -> Option<MetricsSet>; |
60 | | |
61 | | // TODO add desired input ordering |
62 | | // How does this sink want its input ordered? |
63 | | |
64 | | /// Writes the data to the sink, returns the number of values written |
65 | | /// |
66 | | /// This method will be called exactly once during each DML |
67 | | /// statement. Thus prior to return, the sink should do any commit |
68 | | /// or rollback required. |
69 | | async fn write_all( |
70 | | &self, |
71 | | data: SendableRecordBatchStream, |
72 | | context: &Arc<TaskContext>, |
73 | | ) -> Result<u64>; |
74 | | } |
75 | | |
76 | | #[deprecated(since = "38.0.0", note = "Use [`DataSinkExec`] instead")] |
77 | | pub type FileSinkExec = DataSinkExec; |
78 | | |
79 | | /// Execution plan for writing record batches to a [`DataSink`] |
80 | | /// |
81 | | /// Returns a single row with the number of values written |
82 | | pub struct DataSinkExec { |
83 | | /// Input plan that produces the record batches to be written. |
84 | | input: Arc<dyn ExecutionPlan>, |
85 | | /// Sink to which to write |
86 | | sink: Arc<dyn DataSink>, |
87 | | /// Schema of the sink for validating the input data |
88 | | sink_schema: SchemaRef, |
89 | | /// Schema describing the structure of the output data. |
90 | | count_schema: SchemaRef, |
91 | | /// Optional required sort order for output data. |
92 | | sort_order: Option<LexRequirement>, |
93 | | cache: PlanProperties, |
94 | | } |
95 | | |
96 | | impl fmt::Debug for DataSinkExec { |
97 | 0 | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { |
98 | 0 | write!(f, "DataSinkExec schema: {:?}", self.count_schema) |
99 | 0 | } |
100 | | } |
101 | | |
102 | | impl DataSinkExec { |
103 | | /// Create a plan to write to `sink` |
104 | 0 | pub fn new( |
105 | 0 | input: Arc<dyn ExecutionPlan>, |
106 | 0 | sink: Arc<dyn DataSink>, |
107 | 0 | sink_schema: SchemaRef, |
108 | 0 | sort_order: Option<LexRequirement>, |
109 | 0 | ) -> Self { |
110 | 0 | let count_schema = make_count_schema(); |
111 | 0 | let cache = Self::create_schema(&input, count_schema); |
112 | 0 | Self { |
113 | 0 | input, |
114 | 0 | sink, |
115 | 0 | sink_schema, |
116 | 0 | count_schema: make_count_schema(), |
117 | 0 | sort_order, |
118 | 0 | cache, |
119 | 0 | } |
120 | 0 | } |
121 | | |
122 | | /// Input execution plan |
123 | 0 | pub fn input(&self) -> &Arc<dyn ExecutionPlan> { |
124 | 0 | &self.input |
125 | 0 | } |
126 | | |
127 | | /// Returns insert sink |
128 | 0 | pub fn sink(&self) -> &dyn DataSink { |
129 | 0 | self.sink.as_ref() |
130 | 0 | } |
131 | | |
132 | | /// Optional sort order for output data |
133 | 0 | pub fn sort_order(&self) -> &Option<LexRequirement> { |
134 | 0 | &self.sort_order |
135 | 0 | } |
136 | | |
137 | 0 | fn create_schema( |
138 | 0 | input: &Arc<dyn ExecutionPlan>, |
139 | 0 | schema: SchemaRef, |
140 | 0 | ) -> PlanProperties { |
141 | 0 | let eq_properties = EquivalenceProperties::new(schema); |
142 | 0 | PlanProperties::new( |
143 | 0 | eq_properties, |
144 | 0 | Partitioning::UnknownPartitioning(1), |
145 | 0 | input.execution_mode(), |
146 | 0 | ) |
147 | 0 | } |
148 | | } |
149 | | |
150 | | impl DisplayAs for DataSinkExec { |
151 | 0 | fn fmt_as( |
152 | 0 | &self, |
153 | 0 | t: DisplayFormatType, |
154 | 0 | f: &mut std::fmt::Formatter, |
155 | 0 | ) -> std::fmt::Result { |
156 | 0 | match t { |
157 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
158 | 0 | write!(f, "DataSinkExec: sink=")?; |
159 | 0 | self.sink.fmt_as(t, f) |
160 | | } |
161 | | } |
162 | 0 | } |
163 | | } |
164 | | |
165 | | impl ExecutionPlan for DataSinkExec { |
166 | 0 | fn name(&self) -> &'static str { |
167 | 0 | "DataSinkExec" |
168 | 0 | } |
169 | | |
170 | | /// Return a reference to Any that can be used for downcasting |
171 | 0 | fn as_any(&self) -> &dyn Any { |
172 | 0 | self |
173 | 0 | } |
174 | | |
175 | 0 | fn properties(&self) -> &PlanProperties { |
176 | 0 | &self.cache |
177 | 0 | } |
178 | | |
179 | 0 | fn benefits_from_input_partitioning(&self) -> Vec<bool> { |
180 | 0 | // DataSink is responsible for dynamically partitioning its |
181 | 0 | // own input at execution time. |
182 | 0 | vec![false] |
183 | 0 | } |
184 | | |
185 | 0 | fn required_input_distribution(&self) -> Vec<Distribution> { |
186 | 0 | // DataSink is responsible for dynamically partitioning its |
187 | 0 | // own input at execution time, and so requires a single input partition. |
188 | 0 | vec![Distribution::SinglePartition; self.children().len()] |
189 | 0 | } |
190 | | |
191 | 0 | fn required_input_ordering(&self) -> Vec<Option<LexRequirement>> { |
192 | 0 | // The required input ordering is set externally (e.g. by a `ListingTable`). |
193 | 0 | // Otherwise, there is no specific requirement (i.e. `sort_expr` is `None`). |
194 | 0 | vec![self.sort_order.as_ref().cloned()] |
195 | 0 | } |
196 | | |
197 | 0 | fn maintains_input_order(&self) -> Vec<bool> { |
198 | 0 | // Maintains ordering in the sense that the written file will reflect |
199 | 0 | // the ordering of the input. For more context, see: |
200 | 0 | // |
201 | 0 | // https://github.com/apache/datafusion/pull/6354#discussion_r1195284178 |
202 | 0 | vec![true] |
203 | 0 | } |
204 | | |
205 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
206 | 0 | vec![&self.input] |
207 | 0 | } |
208 | | |
209 | 0 | fn with_new_children( |
210 | 0 | self: Arc<Self>, |
211 | 0 | children: Vec<Arc<dyn ExecutionPlan>>, |
212 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
213 | 0 | Ok(Arc::new(Self::new( |
214 | 0 | Arc::clone(&children[0]), |
215 | 0 | Arc::clone(&self.sink), |
216 | 0 | Arc::clone(&self.sink_schema), |
217 | 0 | self.sort_order.clone(), |
218 | 0 | ))) |
219 | 0 | } |
220 | | |
221 | | /// Execute the plan and return a stream of `RecordBatch`es for |
222 | | /// the specified partition. |
223 | 0 | fn execute( |
224 | 0 | &self, |
225 | 0 | partition: usize, |
226 | 0 | context: Arc<TaskContext>, |
227 | 0 | ) -> Result<SendableRecordBatchStream> { |
228 | 0 | if partition != 0 { |
229 | 0 | return internal_err!("DataSinkExec can only be called on partition 0!"); |
230 | 0 | } |
231 | 0 | let data = execute_input_stream( |
232 | 0 | Arc::clone(&self.input), |
233 | 0 | Arc::clone(&self.sink_schema), |
234 | 0 | 0, |
235 | 0 | Arc::clone(&context), |
236 | 0 | )?; |
237 | | |
238 | 0 | let count_schema = Arc::clone(&self.count_schema); |
239 | 0 | let sink = Arc::clone(&self.sink); |
240 | 0 |
|
241 | 0 | let stream = futures::stream::once(async move { |
242 | 0 | sink.write_all(data, &context).await.map(make_count_batch) |
243 | 0 | }) |
244 | 0 | .boxed(); |
245 | 0 |
|
246 | 0 | Ok(Box::pin(RecordBatchStreamAdapter::new( |
247 | 0 | count_schema, |
248 | 0 | stream, |
249 | 0 | ))) |
250 | 0 | } |
251 | | |
252 | | /// Returns the metrics of the underlying [DataSink] |
253 | 0 | fn metrics(&self) -> Option<MetricsSet> { |
254 | 0 | self.sink.metrics() |
255 | 0 | } |
256 | | } |
257 | | |
258 | | /// Create a output record batch with a count |
259 | | /// |
260 | | /// ```text |
261 | | /// +-------+, |
262 | | /// | count |, |
263 | | /// +-------+, |
264 | | /// | 6 |, |
265 | | /// +-------+, |
266 | | /// ``` |
267 | 0 | fn make_count_batch(count: u64) -> RecordBatch { |
268 | 0 | let array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef; |
269 | 0 |
|
270 | 0 | RecordBatch::try_from_iter_with_nullable(vec![("count", array, false)]).unwrap() |
271 | 0 | } |
272 | | |
273 | 0 | fn make_count_schema() -> SchemaRef { |
274 | 0 | // define a schema. |
275 | 0 | Arc::new(Schema::new(vec![Field::new( |
276 | 0 | "count", |
277 | 0 | DataType::UInt64, |
278 | 0 | false, |
279 | 0 | )])) |
280 | 0 | } |