-
Notifications
You must be signed in to change notification settings - Fork 1.2k
/
cross_join.rs
685 lines (609 loc) · 23 KB
/
cross_join.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
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
// 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::{ready, StreamExt};
use futures::{Stream, TryStreamExt};
use std::{any::Any, sync::Arc, task::Poll};
use arrow::datatypes::{Fields, Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet};
use crate::DisplayAs;
use crate::{
coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec,
ColumnStatistics, DisplayFormatType, Distribution, EquivalenceProperties,
ExecutionPlan, Partitioning, PhysicalSortExpr, RecordBatchStream,
SendableRecordBatchStream, Statistics,
};
use async_trait::async_trait;
use datafusion_common::{plan_err, DataFusionError};
use datafusion_common::{Result, ScalarValue};
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation};
use datafusion_execution::TaskContext;
use super::utils::{
adjust_right_output_partitioning, cross_join_equivalence_properties,
BuildProbeJoinMetrics, OnceAsync, OnceFut,
};
/// Data of the left side
type JoinLeftData = (RecordBatch, MemoryReservation);
/// 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
pub left: Arc<dyn ExecutionPlan>,
/// right (probe) side which are combined with left side
pub right: Arc<dyn ExecutionPlan>,
/// The schema once the join is applied
schema: SchemaRef,
/// Build-side data
left_fut: OnceAsync<JoinLeftData>,
/// Execution plan metrics
metrics: ExecutionPlanMetricsSet,
}
impl CrossJoinExec {
/// Create a new [CrossJoinExec].
pub fn new(left: Arc<dyn ExecutionPlan>, right: Arc<dyn ExecutionPlan>) -> Self {
// left then right
let all_columns: Fields = {
let left_schema = left.schema();
let right_schema = right.schema();
let left_fields = left_schema.fields().iter();
let right_fields = right_schema.fields().iter();
left_fields.chain(right_fields).cloned().collect()
};
let schema = Arc::new(Schema::new(all_columns));
CrossJoinExec {
left,
right,
schema,
left_fut: Default::default(),
metrics: ExecutionPlanMetricsSet::default(),
}
}
/// 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
}
}
/// Asynchronously collect the result of the left child
async fn load_left_input(
left: Arc<dyn ExecutionPlan>,
context: Arc<TaskContext>,
metrics: BuildProbeJoinMetrics,
reservation: MemoryReservation,
) -> Result<JoinLeftData> {
// merge all left parts into a single stream
let merge = {
if left.output_partitioning().partition_count() != 1 {
Arc::new(CoalescePartitionsExec::new(left.clone()))
} else {
left.clone()
}
};
let stream = merge.execute(0, context)?;
// Load all batches and count the rows
let (batches, num_rows, _, reservation) = stream
.try_fold(
(Vec::new(), 0usize, metrics, reservation),
|mut acc, batch| async {
let batch_size = batch.get_array_memory_size();
// Reserve memory for incoming batch
acc.3.try_grow(batch_size)?;
// Update metrics
acc.2.build_mem_used.add(batch_size);
acc.2.build_input_batches.add(1);
acc.2.build_input_rows.add(batch.num_rows());
// Update rowcount
acc.1 += batch.num_rows();
// Push batch to output
acc.0.push(batch);
Ok(acc)
},
)
.await?;
let merged_batch = concat_batches(&left.schema(), &batches, num_rows)?;
Ok((merged_batch, reservation))
}
impl DisplayAs for CrossJoinExec {
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(f, "CrossJoinExec")
}
}
}
}
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 metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
/// Specifies whether this plan generates an infinite stream of records.
/// If the plan does not support pipelining, but its input(s) are
/// infinite, returns an error to indicate this.
fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
if children[0] || children[1] {
plan_err!(
"Cross Join Error: Cross join is not supported for the unbounded inputs."
)
} else {
Ok(false)
}
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(CrossJoinExec::new(
children[0].clone(),
children[1].clone(),
)))
}
fn required_input_distribution(&self) -> Vec<Distribution> {
vec![
Distribution::SinglePartition,
Distribution::UnspecifiedDistribution,
]
}
// TODO optimize CrossJoin implementation to generate M * N partitions
fn output_partitioning(&self) -> Partitioning {
let left_columns_len = self.left.schema().fields.len();
adjust_right_output_partitioning(
self.right.output_partitioning(),
left_columns_len,
)
}
// TODO check the output ordering of CrossJoin
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
None
}
fn equivalence_properties(&self) -> EquivalenceProperties {
let left_columns_len = self.left.schema().fields.len();
cross_join_equivalence_properties(
self.left.equivalence_properties(),
self.right.equivalence_properties(),
left_columns_len,
self.schema(),
)
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
let stream = self.right.execute(partition, context.clone())?;
let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics);
// Initialization of operator-level reservation
let reservation =
MemoryConsumer::new("CrossJoinExec").register(context.memory_pool());
let left_fut = self.left_fut.once(|| {
load_left_input(
self.left.clone(),
context,
join_metrics.clone(),
reservation,
)
});
Ok(Box::pin(CrossJoinStream {
schema: self.schema.clone(),
left_fut,
right: stream,
right_batch: Arc::new(parking_lot::Mutex::new(None)),
left_index: 0,
join_metrics,
}))
}
fn statistics(&self) -> Statistics {
stats_cartesian_product(
self.left.statistics(),
self.left.schema().fields().len(),
self.right.statistics(),
self.right.schema().fields().len(),
)
}
}
/// [left/right]_col_count are required in case the column statistics are None
fn stats_cartesian_product(
left_stats: Statistics,
left_col_count: usize,
right_stats: Statistics,
right_col_count: usize,
) -> Statistics {
let left_row_count = left_stats.num_rows;
let right_row_count = right_stats.num_rows;
// calculate global stats
let is_exact = left_stats.is_exact && right_stats.is_exact;
let num_rows = left_stats
.num_rows
.zip(right_stats.num_rows)
.map(|(a, b)| a * b);
// the result size is two times a*b because you have the columns of both left and right
let total_byte_size = left_stats
.total_byte_size
.zip(right_stats.total_byte_size)
.map(|(a, b)| 2 * a * b);
// calculate column stats
let column_statistics =
// complete the column statistics if they are missing only on one side
match (left_stats.column_statistics, right_stats.column_statistics) {
(None, None) => None,
(None, Some(right_col_stat)) => Some((
vec![ColumnStatistics::default(); left_col_count],
right_col_stat,
)),
(Some(left_col_stat), None) => Some((
left_col_stat,
vec![ColumnStatistics::default(); right_col_count],
)),
(Some(left_col_stat), Some(right_col_stat)) => {
Some((left_col_stat, right_col_stat))
}
}
.map(|(left_col_stats, right_col_stats)| {
// the null counts must be multiplied by the row counts of the other side (if defined)
// Min, max and distinct_count on the other hand are invariants.
left_col_stats.into_iter().map(|s| ColumnStatistics{
null_count: s.null_count.zip(right_row_count).map(|(a, b)| a * b),
distinct_count: s.distinct_count,
min_value: s.min_value,
max_value: s.max_value,
}).chain(
right_col_stats.into_iter().map(|s| ColumnStatistics{
null_count: s.null_count.zip(left_row_count).map(|(a, b)| a * b),
distinct_count: s.distinct_count,
min_value: s.min_value,
max_value: s.max_value,
})).collect()
});
Statistics {
is_exact,
num_rows,
total_byte_size,
column_statistics,
}
}
/// A stream that issues [RecordBatch]es as they arrive from the right of the join.
struct CrossJoinStream {
/// Input schema
schema: Arc<Schema>,
/// future for data from left side
left_fut: OnceFut<JoinLeftData>,
/// right
right: SendableRecordBatchStream,
/// Current value on the left
left_index: usize,
/// Current batch being processed from the right side
right_batch: Arc<parking_lot::Mutex<Option<RecordBatch>>>,
/// join execution metrics
join_metrics: BuildProbeJoinMetrics,
}
impl RecordBatchStream for CrossJoinStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
fn build_batch(
left_index: usize,
batch: &RecordBatch,
left_data: &RecordBatch,
schema: &Schema,
) -> Result<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<_>>>()?;
RecordBatch::try_new(
Arc::new(schema.clone()),
arrays
.iter()
.chain(batch.columns().iter())
.cloned()
.collect(),
)
.map_err(Into::into)
}
#[async_trait]
impl Stream for CrossJoinStream {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Option<Self::Item>> {
self.poll_next_impl(cx)
}
}
impl CrossJoinStream {
/// Separate implementation function that unpins the [`CrossJoinStream`] so
/// that partial borrows work correctly
fn poll_next_impl(
&mut self,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Option<Result<RecordBatch>>> {
let build_timer = self.join_metrics.build_time.timer();
let (left_data, _) = match ready!(self.left_fut.get(cx)) {
Ok(left_data) => left_data,
Err(e) => return Poll::Ready(Some(Err(e))),
};
build_timer.done();
if left_data.num_rows() == 0 {
return Poll::Ready(None);
}
if self.left_index > 0 && self.left_index < left_data.num_rows() {
let join_timer = self.join_metrics.join_time.timer();
let right_batch = {
let right_batch = self.right_batch.lock();
right_batch.clone().unwrap()
};
let result =
build_batch(self.left_index, &right_batch, left_data, &self.schema);
self.join_metrics.input_rows.add(right_batch.num_rows());
if let Ok(ref batch) = result {
join_timer.done();
self.join_metrics.output_batches.add(1);
self.join_metrics.output_rows.add(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 join_timer = self.join_metrics.join_time.timer();
let result =
build_batch(self.left_index, &batch, left_data, &self.schema);
self.join_metrics.input_batches.add(1);
self.join_metrics.input_rows.add(batch.num_rows());
if let Ok(ref batch) = result {
join_timer.done();
self.join_metrics.output_batches.add(1);
self.join_metrics.output_rows.add(batch.num_rows());
}
self.left_index = 1;
let mut right_batch = self.right_batch.lock();
*right_batch = Some(batch);
Some(result)
}
other => other,
})
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::common;
use crate::test::build_table_scan_i32;
use datafusion_common::{assert_batches_sorted_eq, assert_contains};
use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv};
async fn join_collect(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
context: Arc<TaskContext>,
) -> Result<(Vec<String>, Vec<RecordBatch>)> {
let join = CrossJoinExec::new(left, right);
let columns_header = columns(&join.schema());
let stream = join.execute(0, context)?;
let batches = common::collect(stream).await?;
Ok((columns_header, batches))
}
#[tokio::test]
async fn test_stats_cartesian_product() {
let left_row_count = 11;
let left_bytes = 23;
let right_row_count = 7;
let right_bytes = 27;
let left = Statistics {
is_exact: true,
num_rows: Some(left_row_count),
total_byte_size: Some(left_bytes),
column_statistics: Some(vec![
ColumnStatistics {
distinct_count: Some(5),
max_value: Some(ScalarValue::Int64(Some(21))),
min_value: Some(ScalarValue::Int64(Some(-4))),
null_count: Some(0),
},
ColumnStatistics {
distinct_count: Some(1),
max_value: Some(ScalarValue::Utf8(Some(String::from("x")))),
min_value: Some(ScalarValue::Utf8(Some(String::from("a")))),
null_count: Some(3),
},
]),
};
let right = Statistics {
is_exact: true,
num_rows: Some(right_row_count),
total_byte_size: Some(right_bytes),
column_statistics: Some(vec![ColumnStatistics {
distinct_count: Some(3),
max_value: Some(ScalarValue::Int64(Some(12))),
min_value: Some(ScalarValue::Int64(Some(0))),
null_count: Some(2),
}]),
};
let result = stats_cartesian_product(left, 3, right, 2);
let expected = Statistics {
is_exact: true,
num_rows: Some(left_row_count * right_row_count),
total_byte_size: Some(2 * left_bytes * right_bytes),
column_statistics: Some(vec![
ColumnStatistics {
distinct_count: Some(5),
max_value: Some(ScalarValue::Int64(Some(21))),
min_value: Some(ScalarValue::Int64(Some(-4))),
null_count: Some(0),
},
ColumnStatistics {
distinct_count: Some(1),
max_value: Some(ScalarValue::Utf8(Some(String::from("x")))),
min_value: Some(ScalarValue::Utf8(Some(String::from("a")))),
null_count: Some(3 * right_row_count),
},
ColumnStatistics {
distinct_count: Some(3),
max_value: Some(ScalarValue::Int64(Some(12))),
min_value: Some(ScalarValue::Int64(Some(0))),
null_count: Some(2 * left_row_count),
},
]),
};
assert_eq!(result, expected);
}
#[tokio::test]
async fn test_stats_cartesian_product_with_unknwon_size() {
let left_row_count = 11;
let left = Statistics {
is_exact: true,
num_rows: Some(left_row_count),
total_byte_size: Some(23),
column_statistics: Some(vec![
ColumnStatistics {
distinct_count: Some(5),
max_value: Some(ScalarValue::Int64(Some(21))),
min_value: Some(ScalarValue::Int64(Some(-4))),
null_count: Some(0),
},
ColumnStatistics {
distinct_count: Some(1),
max_value: Some(ScalarValue::Utf8(Some(String::from("x")))),
min_value: Some(ScalarValue::Utf8(Some(String::from("a")))),
null_count: Some(3),
},
]),
};
let right = Statistics {
is_exact: true,
num_rows: None, // not defined!
total_byte_size: None, // not defined!
column_statistics: Some(vec![ColumnStatistics {
distinct_count: Some(3),
max_value: Some(ScalarValue::Int64(Some(12))),
min_value: Some(ScalarValue::Int64(Some(0))),
null_count: Some(2),
}]),
};
let result = stats_cartesian_product(left, 3, right, 2);
let expected = Statistics {
is_exact: true,
num_rows: None,
total_byte_size: None,
column_statistics: Some(vec![
ColumnStatistics {
distinct_count: Some(5),
max_value: Some(ScalarValue::Int64(Some(21))),
min_value: Some(ScalarValue::Int64(Some(-4))),
null_count: None, // we don't know the row count on the right
},
ColumnStatistics {
distinct_count: Some(1),
max_value: Some(ScalarValue::Utf8(Some(String::from("x")))),
min_value: Some(ScalarValue::Utf8(Some(String::from("a")))),
null_count: None, // we don't know the row count on the right
},
ColumnStatistics {
distinct_count: Some(3),
max_value: Some(ScalarValue::Int64(Some(12))),
min_value: Some(ScalarValue::Int64(Some(0))),
null_count: Some(2 * left_row_count),
},
]),
};
assert_eq!(result, expected);
}
#[tokio::test]
async fn test_join() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table_scan_i32(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 6]),
("c1", &vec![7, 8, 9]),
);
let right = build_table_scan_i32(
("a2", &vec![10, 11]),
("b2", &vec![12, 13]),
("c2", &vec![14, 15]),
);
let (columns, batches) = join_collect(left, right, task_ctx).await?;
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b2 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | 10 | 12 | 14 |",
"| 1 | 4 | 7 | 11 | 13 | 15 |",
"| 2 | 5 | 8 | 10 | 12 | 14 |",
"| 2 | 5 | 8 | 11 | 13 | 15 |",
"| 3 | 6 | 9 | 10 | 12 | 14 |",
"| 3 | 6 | 9 | 11 | 13 | 15 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn test_overallocation() -> Result<()> {
let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0);
let runtime = Arc::new(RuntimeEnv::new(runtime_config)?);
let task_ctx = TaskContext::default().with_runtime(runtime);
let task_ctx = Arc::new(task_ctx);
let left = build_table_scan_i32(
("a1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
("b1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
("c1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
);
let right = build_table_scan_i32(
("a2", &vec![10, 11]),
("b2", &vec![12, 13]),
("c2", &vec![14, 15]),
);
let err = join_collect(left, right, task_ctx).await.unwrap_err();
assert_contains!(
err.to_string(),
"External error: Resources exhausted: Failed to allocate additional"
);
assert_contains!(err.to_string(), "CrossJoinExec");
Ok(())
}
/// Returns the column names on the schema
fn columns(schema: &Schema) -> Vec<String> {
schema.fields().iter().map(|f| f.name().clone()).collect()
}
}