Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
30 changes: 30 additions & 0 deletions datafusion/core/tests/sql/explain_analyze.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1103,3 +1103,33 @@ async fn csv_explain_analyze_with_statistics() {
", statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]]"
);
}

#[tokio::test]
async fn nested_loop_join_selectivity() {
for (join_type, expected_selectivity) in [
("INNER", "1% (1/100)"),
("LEFT", "10% (10/100)"),
("RIGHT", "10% (10/100)"),
// 1 match + 9 left + 9 right = 19
("FULL", "19% (19/100)"),
] {
let ctx = SessionContext::new();
let sql = format!(
"EXPLAIN ANALYZE SELECT * \
FROM generate_series(1, 10) as t1(a) \
{join_type} JOIN generate_series(1, 10) as t2(b) \
ON (t1.a + t2.b) = 20"
);

let actual = execute_to_batches(&ctx, sql.as_str()).await;
let formatted = arrow::util::pretty::pretty_format_batches(&actual)
.unwrap()
.to_string();

assert_metrics!(
&formatted,
"NestedLoopJoinExec",
&format!("selectivity={expected_selectivity}")
);
}
}
78 changes: 56 additions & 22 deletions datafusion/physical-plan/src/joins/nested_loop_join.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,9 @@ use crate::joins::utils::{
OnceAsync, OnceFut,
};
use crate::joins::SharedBitmapBuilder;
use crate::metrics::{Count, ExecutionPlanMetricsSet, MetricsSet};
use crate::metrics::{
Count, ExecutionPlanMetricsSet, MetricBuilder, MetricType, MetricsSet, RatioMetrics,
};
use crate::projection::{
try_embed_projection, try_pushdown_through_join, EmbeddedProjection, JoinData,
ProjectionExec,
Expand Down Expand Up @@ -496,7 +498,7 @@ impl ExecutionPlan for NestedLoopJoinExec {
);
}

let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics);
let metrics = NestedLoopJoinMetrics::new(&self.metrics, partition);

// Initialization reservation for load of inner table
let load_reservation =
Expand All @@ -508,7 +510,7 @@ impl ExecutionPlan for NestedLoopJoinExec {

Ok(collect_left_input(
stream,
join_metrics.clone(),
metrics.join_metrics.clone(),
load_reservation,
need_produce_result_in_final(self.join_type),
self.right().output_partitioning().partition_count(),
Expand All @@ -535,7 +537,7 @@ impl ExecutionPlan for NestedLoopJoinExec {
probe_side_data,
build_side_data,
column_indices_after_projection,
join_metrics,
metrics,
batch_size,
)))
}
Expand Down Expand Up @@ -749,7 +751,7 @@ pub(crate) struct NestedLoopJoinStream {
/// the join filter (e.g., `JOIN ON (b+c)>0`).
pub(crate) column_indices: Vec<ColumnIndex>,
/// Join execution metrics
pub(crate) join_metrics: BuildProbeJoinMetrics,
pub(crate) metrics: NestedLoopJoinMetrics,

/// `batch_size` from configuration
batch_size: usize,
Expand Down Expand Up @@ -794,6 +796,24 @@ pub(crate) struct NestedLoopJoinStream {
current_right_batch_matched: Option<BooleanArray>,
}

pub(crate) struct NestedLoopJoinMetrics {
/// Join execution metrics
pub(crate) join_metrics: BuildProbeJoinMetrics,
/// Selectivity of the join: output_rows / (left_rows * right_rows)
pub(crate) selectivity: RatioMetrics,
}

impl NestedLoopJoinMetrics {
pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self {
Self {
join_metrics: BuildProbeJoinMetrics::new(partition, metrics),
selectivity: MetricBuilder::new(metrics)
.with_type(MetricType::SUMMARY)
.ratio_metrics("selectivity", partition),
}
}
}

impl Stream for NestedLoopJoinStream {
type Item = Result<RecordBatch>;

Expand Down Expand Up @@ -844,7 +864,7 @@ impl Stream for NestedLoopJoinStream {
// -side batches), related metrics except build time will be
// updated.
// stop on drop
let build_metric = self.join_metrics.build_time.clone();
let build_metric = self.metrics.join_metrics.build_time.clone();
let _build_timer = build_metric.timer();

match self.handle_buffering_left(cx) {
Expand Down Expand Up @@ -878,7 +898,7 @@ impl Stream for NestedLoopJoinStream {
NLJState::FetchingRight => {
debug!("[NLJState] Entering: {:?}", self.state);
// stop on drop
let join_metric = self.join_metrics.join_time.clone();
let join_metric = self.metrics.join_metrics.join_time.clone();
let _join_timer = join_metric.timer();

match self.handle_fetching_right(cx) {
Expand All @@ -905,13 +925,13 @@ impl Stream for NestedLoopJoinStream {
debug!("[NLJState] Entering: {:?}", self.state);

// stop on drop
let join_metric = self.join_metrics.join_time.clone();
let join_metric = self.metrics.join_metrics.join_time.clone();
let _join_timer = join_metric.timer();

match self.handle_probe_right() {
ControlFlow::Continue(()) => continue,
ControlFlow::Break(poll) => {
return self.join_metrics.baseline.record_poll(poll)
return self.metrics.join_metrics.baseline.record_poll(poll)
}
}
}
Expand All @@ -926,13 +946,13 @@ impl Stream for NestedLoopJoinStream {
debug!("[NLJState] Entering: {:?}", self.state);

// stop on drop
let join_metric = self.join_metrics.join_time.clone();
let join_metric = self.metrics.join_metrics.join_time.clone();
let _join_timer = join_metric.timer();

match self.handle_emit_right_unmatched() {
ControlFlow::Continue(()) => continue,
ControlFlow::Break(poll) => {
return self.join_metrics.baseline.record_poll(poll)
return self.metrics.join_metrics.baseline.record_poll(poll)
}
}
}
Expand All @@ -956,13 +976,13 @@ impl Stream for NestedLoopJoinStream {
debug!("[NLJState] Entering: {:?}", self.state);

// stop on drop
let join_metric = self.join_metrics.join_time.clone();
let join_metric = self.metrics.join_metrics.join_time.clone();
let _join_timer = join_metric.timer();

match self.handle_emit_left_unmatched() {
ControlFlow::Continue(()) => continue,
ControlFlow::Break(poll) => {
return self.join_metrics.baseline.record_poll(poll)
return self.metrics.join_metrics.baseline.record_poll(poll)
}
}
}
Expand All @@ -972,13 +992,13 @@ impl Stream for NestedLoopJoinStream {
debug!("[NLJState] Entering: {:?}", self.state);

// stop on drop
let join_metric = self.join_metrics.join_time.clone();
let join_metric = self.metrics.join_metrics.join_time.clone();
let _join_timer = join_metric.timer();
// counting it in join timer due to there might be some
// final resout batches to output in this state

let poll = self.handle_done();
return self.join_metrics.baseline.record_poll(poll);
return self.metrics.join_metrics.baseline.record_poll(poll);
}
}
}
Expand All @@ -1000,7 +1020,7 @@ impl NestedLoopJoinStream {
right_data: SendableRecordBatchStream,
left_data: OnceFut<JoinLeftData>,
column_indices: Vec<ColumnIndex>,
join_metrics: BuildProbeJoinMetrics,
metrics: NestedLoopJoinMetrics,
batch_size: usize,
) -> Self {
Self {
Expand All @@ -1010,7 +1030,7 @@ impl NestedLoopJoinStream {
right_data,
column_indices,
left_data,
join_metrics,
metrics,
buffered_left_data: None,
output_buffer: Box::new(BatchCoalescer::new(schema, batch_size)),
batch_size,
Expand Down Expand Up @@ -1057,8 +1077,8 @@ impl NestedLoopJoinStream {
Some(Ok(right_batch)) => {
// Update metrics
let right_batch_size = right_batch.num_rows();
self.join_metrics.input_rows.add(right_batch_size);
self.join_metrics.input_batches.add(1);
self.metrics.join_metrics.input_rows.add(right_batch_size);
self.metrics.join_metrics.input_batches.add(1);

// Skip the empty batch
if right_batch_size == 0 {
Expand Down Expand Up @@ -1108,6 +1128,17 @@ impl NestedLoopJoinStream {
Ok(false) => {
// Left exhausted, transition to FetchingRight
self.left_probe_idx = 0;

// Selectivity Metric: Update total possibilities for the batch (left_rows * right_rows)
// If memory-limited execution is implemented, this logic must be updated accordingly.
if let (Ok(left_data), Some(right_batch)) =
(self.get_left_data(), self.current_right_batch.as_ref())
{
let left_rows = left_data.batch().num_rows();
let right_rows = right_batch.num_rows();
self.metrics.selectivity.add_total(left_rows * right_rows);
}

if self.should_track_unmatched_right {
debug_assert!(
self.current_right_batch_matched.is_some(),
Expand Down Expand Up @@ -1138,7 +1169,6 @@ impl NestedLoopJoinStream {
&& self.current_right_batch.is_some(),
"This state is yielding output for unmatched rows in the current right batch, so both the right batch and the bitmap must be present"
);

// Construct the result batch for unmatched right rows using a utility function
match self.process_right_unmatched() {
Ok(Some(batch)) => {
Expand Down Expand Up @@ -1205,7 +1235,7 @@ impl NestedLoopJoinStream {
// should be with the expected schema for this operator
if !self.handled_empty_output {
let zero_count = Count::new();
if *self.join_metrics.baseline.output_rows() == zero_count {
if *self.metrics.join_metrics.baseline.output_rows() == zero_count {
let empty_batch = RecordBatch::new_empty(Arc::clone(&self.output_schema));
self.handled_empty_output = true;
return Poll::Ready(Some(Ok(empty_batch)));
Expand Down Expand Up @@ -1455,7 +1485,11 @@ impl NestedLoopJoinStream {
if let Some(batch) = self.output_buffer.next_completed_batch() {
// HACK: this is not part of `BaselineMetrics` yet, so update it
// manually
self.join_metrics.output_batches.add(1);
self.metrics.join_metrics.output_batches.add(1);

// Update output rows for selectivity metric
let output_rows = batch.num_rows();
self.metrics.selectivity.add_part(output_rows);
Comment on lines +1488 to +1492
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Update output_rows at flush time.


return Some(Poll::Ready(Some(Ok(batch))));
}
Expand Down