forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 5
Expand file tree
/
Copy pathcross_join.rs
More file actions
333 lines (295 loc) · 10.6 KB
/
cross_join.rs
File metadata and controls
333 lines (295 loc) · 10.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
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! Defines the cross join plan for loading the left side of the cross join
//! and producing batches in parallel for the right partitions
use futures::{lock::Mutex, StreamExt};
use std::{any::Any, sync::Arc, task::Poll};
use arrow::datatypes::{Schema, SchemaRef};
use arrow::error::Result as ArrowResult;
use arrow::record_batch::RecordBatch;
use futures::{Stream, TryStreamExt};
use super::{
coalesce_partitions::CoalescePartitionsExec, hash_utils::check_join_is_valid,
};
use crate::{
error::{DataFusionError, Result},
scalar::ScalarValue,
};
use async_trait::async_trait;
use std::time::Instant;
use super::{
coalesce_batches::concat_batches, memory::MemoryStream, DisplayFormatType,
ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream,
};
use log::debug;
/// Data of the left side
type JoinLeftData = RecordBatch;
/// executes partitions in parallel and combines them into a set of
/// partitions by combining all values from the left with all values on the right
#[derive(Debug)]
pub struct CrossJoinExec {
/// left (build) side which gets loaded in memory
left: Arc<dyn ExecutionPlan>,
/// right (probe) side which are combined with left side
right: Arc<dyn ExecutionPlan>,
/// The schema once the join is applied
schema: SchemaRef,
/// Build-side data
build_side: Arc<Mutex<Option<JoinLeftData>>>,
}
impl CrossJoinExec {
/// Tries to create a new [CrossJoinExec].
/// # Error
/// This function errors when left and right schema's can't be combined
pub fn try_new(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
) -> Result<Self> {
let left_schema = left.schema();
let right_schema = right.schema();
check_join_is_valid(&left_schema, &right_schema, &[])?;
let left_schema = left.schema();
let left_fields = left_schema.fields().iter();
let right_schema = right.schema();
let right_fields = right_schema.fields().iter();
// left then right
let all_columns = left_fields.chain(right_fields).cloned().collect();
let schema = Arc::new(Schema::new(all_columns));
Ok(CrossJoinExec {
left,
right,
schema,
build_side: Arc::new(Mutex::new(None)),
})
}
/// left (build) side which gets loaded in memory
pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
&self.left
}
/// right side which gets combined with left side
pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
&self.right
}
}
#[async_trait]
impl ExecutionPlan for CrossJoinExec {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
vec![self.left.clone(), self.right.clone()]
}
fn with_new_children(
&self,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
match children.len() {
2 => Ok(Arc::new(CrossJoinExec::try_new(
children[0].clone(),
children[1].clone(),
)?)),
_ => Err(DataFusionError::Internal(
"CrossJoinExec wrong number of children".to_string(),
)),
}
}
fn output_partitioning(&self) -> Partitioning {
self.right.output_partitioning()
}
async fn execute(&self, partition: usize) -> Result<SendableRecordBatchStream> {
// we only want to compute the build side once
let left_data = {
let mut build_side = self.build_side.lock().await;
match build_side.as_ref() {
Some(stream) => stream.clone(),
None => {
let start = Instant::now();
// merge all left parts into a single stream
let merge = CoalescePartitionsExec::new(self.left.clone());
let stream = merge.execute(0).await?;
// Load all batches and count the rows
let (batches, num_rows) = stream
.try_fold((Vec::new(), 0usize), |mut acc, batch| async {
acc.1 += batch.num_rows();
acc.0.push(batch);
Ok(acc)
})
.await?;
let merged_batch =
concat_batches(&self.left.schema(), &batches, num_rows)?;
*build_side = Some(merged_batch.clone());
debug!(
"Built build-side of cross join containing {} rows in {} ms",
num_rows,
start.elapsed().as_millis()
);
merged_batch
}
}
};
let stream = self.right.execute(partition).await?;
if left_data.num_rows() == 0 {
return Ok(Box::pin(MemoryStream::try_new(
vec![],
self.schema.clone(),
None,
)?));
}
Ok(Box::pin(CrossJoinStream {
schema: self.schema.clone(),
left_data,
right: stream,
right_batch: Arc::new(std::sync::Mutex::new(None)),
left_index: 0,
num_input_batches: 0,
num_input_rows: 0,
num_output_batches: 0,
num_output_rows: 0,
join_time: 0,
}))
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default => {
write!(f, "CrossJoinExec")
}
}
}
}
/// A stream that issues [RecordBatch]es as they arrive from the right of the join.
struct CrossJoinStream {
/// Input schema
schema: Arc<Schema>,
/// data from the left side
left_data: JoinLeftData,
/// right
right: SendableRecordBatchStream,
/// Current value on the left
left_index: usize,
/// Current batch being processed from the right side
right_batch: Arc<std::sync::Mutex<Option<RecordBatch>>>,
/// number of input batches
num_input_batches: usize,
/// number of input rows
num_input_rows: usize,
/// number of batches produced
num_output_batches: usize,
/// number of rows produced
num_output_rows: usize,
/// total time for joining probe-side batches to the build-side batches
join_time: usize,
}
impl RecordBatchStream for CrossJoinStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
fn build_batch(
left_index: usize,
batch: &RecordBatch,
left_data: &RecordBatch,
schema: &Schema,
) -> ArrowResult<RecordBatch> {
// Repeat value on the left n times
let arrays = left_data
.columns()
.iter()
.map(|arr| {
let scalar = ScalarValue::try_from_array(arr, left_index)?;
Ok(scalar.to_array_of_size(batch.num_rows()))
})
.collect::<Result<Vec<_>>>()
.map_err(|x| x.into_arrow_external_error())?;
RecordBatch::try_new(
Arc::new(schema.clone()),
arrays
.iter()
.chain(batch.columns().iter())
.cloned()
.collect(),
)
}
#[async_trait]
impl Stream for CrossJoinStream {
type Item = ArrowResult<RecordBatch>;
fn poll_next(
mut self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Option<Self::Item>> {
if self.left_index > 0 && self.left_index < self.left_data.num_rows() {
let start = Instant::now();
let right_batch = {
let right_batch = self.right_batch.lock().unwrap();
right_batch.clone().unwrap()
};
let result =
build_batch(self.left_index, &right_batch, &self.left_data, &self.schema);
self.num_input_rows += right_batch.num_rows();
if let Ok(ref batch) = result {
self.join_time += start.elapsed().as_millis() as usize;
self.num_output_batches += 1;
self.num_output_rows += batch.num_rows();
}
self.left_index += 1;
return Poll::Ready(Some(result));
}
self.left_index = 0;
self.right
.poll_next_unpin(cx)
.map(|maybe_batch| match maybe_batch {
Some(Ok(batch)) => {
let start = Instant::now();
let result = build_batch(
self.left_index,
&batch,
&self.left_data,
&self.schema,
);
self.num_input_batches += 1;
self.num_input_rows += batch.num_rows();
if let Ok(ref batch) = result {
self.join_time += start.elapsed().as_millis() as usize;
self.num_output_batches += 1;
self.num_output_rows += batch.num_rows();
}
self.left_index = 1;
let mut right_batch = self.right_batch.lock().unwrap();
*right_batch = Some(batch);
Some(result)
}
other => {
debug!(
"Processed {} probe-side input batches containing {} rows and \
produced {} output batches containing {} rows in {} ms",
self.num_input_batches,
self.num_input_rows,
self.num_output_batches,
self.num_output_rows,
self.join_time
);
other
}
})
}
}