-
Notifications
You must be signed in to change notification settings - Fork 691
feat(streaming): support locality enforcement and locality backfill #23275
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
71a86b4
b8a76df
6885136
58fe6d8
9b36d47
97252b0
dc18e6c
027d22f
f52a660
2c4bb95
2f50b59
124e8c8
c806a4d
3d5fc58
0d97562
f145ed1
00f7f2b
60bc6f7
7bae65c
171e124
69c7afb
4a86f92
6c6e2f0
1151be1
714defc
c348fe2
8b01b5a
deb2ccb
1d7564f
22bc8b8
16f6b9c
e44a12f
9dabe6a
3dd2398
6857478
679e2b1
a7176c0
a49a912
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,35 @@ | ||
statement ok | ||
set enable_locality_backfill=true; | ||
|
||
statement ok | ||
create table t1(a int, b int); | ||
|
||
statement ok | ||
create table t2(a int, b int); | ||
|
||
statement ok | ||
insert into t1 select i, 123 from generate_series(1, 1000, 1) i; | ||
|
||
statement ok | ||
insert into t2 select i, 123 from generate_series(1, 1000, 1) i ; | ||
|
||
statement ok | ||
flush; | ||
|
||
statement ok | ||
create materialized view mv as select count(*) from t1 join t2 on t1.a = t2.a group by t1.b; | ||
|
||
|
||
query ? | ||
select * from mv; | ||
---- | ||
1000 | ||
|
||
statement ok | ||
drop materialized view mv; | ||
|
||
statement ok | ||
drop table t1; | ||
|
||
statement ok | ||
drop table t2; |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,19 @@ | ||
- sql: | | ||
set enable_locality_backfill = true; | ||
create table t (a int, b int, c int); | ||
select count(*) from t group by b; | ||
expected_outputs: | ||
- stream_plan | ||
- sql: | | ||
set enable_locality_backfill = true; | ||
create table t1 (a int, b int, c int); | ||
create table t2 (a int, b int, c int); | ||
select count(*) from t1 join t2 on t1.a = t2.a group by t1.b; | ||
expected_outputs: | ||
- stream_plan | ||
- sql: | | ||
set enable_locality_backfill = true; | ||
create table t (a int, b int, c int, primary key (b, a)); | ||
select count(*) from t group by a, b; | ||
expected_outputs: | ||
- stream_plan |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,43 @@ | ||
# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. | ||
chenzl25 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
- sql: | | ||
set enable_locality_backfill = true; | ||
create table t (a int, b int, c int); | ||
select count(*) from t group by b; | ||
stream_plan: |- | ||
StreamMaterialize { columns: [count, t.b(hidden)], stream_key: [t.b], pk_columns: [t.b], pk_conflict: NoCheck } | ||
└─StreamProject { exprs: [count, t.b] } | ||
└─StreamHashAgg { group_key: [t.b], aggs: [count] } | ||
└─StreamLocalityProvider { locality_columns: [t.b] } | ||
└─StreamExchange { dist: HashShard(t.b) } | ||
└─StreamTableScan { table: t, columns: [t.b, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } | ||
- sql: | | ||
set enable_locality_backfill = true; | ||
create table t1 (a int, b int, c int); | ||
create table t2 (a int, b int, c int); | ||
select count(*) from t1 join t2 on t1.a = t2.a group by t1.b; | ||
stream_plan: |- | ||
StreamMaterialize { columns: [count, t1.b(hidden)], stream_key: [t1.b], pk_columns: [t1.b], pk_conflict: NoCheck } | ||
└─StreamProject { exprs: [count, t1.b] } | ||
└─StreamHashAgg { group_key: [t1.b], aggs: [count] } | ||
└─StreamLocalityProvider { locality_columns: [t1.b] } | ||
└─StreamExchange { dist: HashShard(t1.b) } | ||
└─StreamHashJoin { type: Inner, predicate: t1.a = t2.a, output: [t1.b, t1._row_id, t1.a, t2._row_id] } | ||
├─StreamExchange { dist: HashShard(t1.a) } | ||
│ └─StreamLocalityProvider { locality_columns: [t1.a] } | ||
│ └─StreamExchange { dist: HashShard(t1.a) } | ||
│ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } | ||
└─StreamExchange { dist: HashShard(t2.a) } | ||
└─StreamLocalityProvider { locality_columns: [t2.a] } | ||
└─StreamExchange { dist: HashShard(t2.a) } | ||
└─StreamTableScan { table: t2, columns: [t2.a, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } | ||
- sql: | | ||
set enable_locality_backfill = true; | ||
create table t (a int, b int, c int, primary key (b, a)); | ||
select count(*) from t group by a, b; | ||
stream_plan: |- | ||
StreamMaterialize { columns: [count, t.a(hidden), t.b(hidden)], stream_key: [t.a, t.b], pk_columns: [t.a, t.b], pk_conflict: NoCheck } | ||
└─StreamProject { exprs: [count, t.a, t.b] } | ||
└─StreamHashAgg { group_key: [t.a, t.b], aggs: [count] } | ||
└─StreamLocalityProvider { locality_columns: [t.a, t.b] } | ||
└─StreamExchange [no_shuffle] { dist: UpstreamHashShard(t.b, t.a) } | ||
└─StreamTableScan { table: t, columns: [t.a, t.b], stream_scan_type: ArrangementBackfill, stream_key: [t.b, t.a], pk: [b, a], dist: UpstreamHashShard(t.b, t.a) } |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,78 @@ | ||
// Copyright 2025 RisingWave Labs | ||
// | ||
// Licensed 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. | ||
|
||
use pretty_xmlish::Pretty; | ||
use risingwave_common::catalog::{FieldDisplay, Schema}; | ||
|
||
use super::{GenericPlanNode, GenericPlanRef, impl_distill_unit_from_fields}; | ||
use crate::expr::ExprRewriter; | ||
use crate::optimizer::optimizer_context::OptimizerContextRef; | ||
use crate::optimizer::property::FunctionalDependencySet; | ||
|
||
/// `LocalityProvider` provides locality for operators during backfilling. | ||
/// It buffers input data into a state table using locality columns as primary key prefix. | ||
#[derive(Debug, Clone, PartialEq, Eq, Hash)] | ||
pub struct LocalityProvider<PlanRef> { | ||
pub input: PlanRef, | ||
/// Columns that define the locality | ||
pub locality_columns: Vec<usize>, | ||
} | ||
|
||
impl<PlanRef: GenericPlanRef> LocalityProvider<PlanRef> { | ||
pub fn new(input: PlanRef, locality_columns: Vec<usize>) -> Self { | ||
Self { | ||
input, | ||
locality_columns, | ||
} | ||
} | ||
|
||
pub fn fields_pretty<'a>(&self) -> Vec<(&'a str, Pretty<'a>)> { | ||
let locality_columns_display = self | ||
.locality_columns | ||
.iter() | ||
.map(|&i| Pretty::display(&FieldDisplay(self.input.schema().fields.get(i).unwrap()))) | ||
.collect(); | ||
vec![("locality_columns", Pretty::Array(locality_columns_display))] | ||
} | ||
} | ||
|
||
impl<PlanRef: GenericPlanRef> GenericPlanNode for LocalityProvider<PlanRef> { | ||
fn schema(&self) -> Schema { | ||
self.input.schema().clone() | ||
} | ||
|
||
fn stream_key(&self) -> Option<Vec<usize>> { | ||
Some(self.input.stream_key()?.to_vec()) | ||
} | ||
|
||
fn ctx(&self) -> OptimizerContextRef { | ||
self.input.ctx() | ||
} | ||
|
||
fn functional_dependency(&self) -> FunctionalDependencySet { | ||
self.input.functional_dependency().clone() | ||
} | ||
} | ||
|
||
impl<PlanRef: GenericPlanRef> LocalityProvider<PlanRef> { | ||
pub fn rewrite_exprs(&mut self, _r: &mut dyn ExprRewriter) { | ||
// LocalityProvider doesn't contain expressions to rewrite | ||
} | ||
|
||
pub fn visit_exprs(&self, _v: &mut dyn crate::expr::ExprVisitor) { | ||
// LocalityProvider doesn't contain expressions to visit | ||
} | ||
} | ||
|
||
impl_distill_unit_from_fields!(LocalityProvider, GenericPlanRef); |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,9 +28,9 @@ use super::generic::{ | |
}; | ||
use super::utils::{Distill, childless_record}; | ||
use super::{ | ||
BatchPlanRef, ColPrunable, ExprRewritable, Logical, LogicalPlanRef as PlanRef, PlanBase, | ||
PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, StreamPlanRef, StreamProject, ToBatch, | ||
ToStream, generic, | ||
BatchPlanRef, ColPrunable, ExprRewritable, Logical, LogicalLocalityProvider, | ||
LogicalPlanRef as PlanRef, PlanBase, PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, | ||
StreamPlanRef, StreamProject, ToBatch, ToStream, generic, | ||
}; | ||
use crate::error::{ErrorCode, Result, RwError}; | ||
use crate::expr::{CollectInputRef, Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, InputRef}; | ||
|
@@ -1541,6 +1541,26 @@ impl LogicalJoin { | |
.into()), | ||
} | ||
} | ||
|
||
fn try_better_locality_inner(&self, columns: &[usize]) -> Option<PlanRef> { | ||
let mut ctx = ToStreamContext::new(false); | ||
// only pass through the locality information if it can be converted to dynamic filter | ||
if let Ok(Some(_)) = self.to_stream_dynamic_filter(self.on().clone(), &mut ctx) { | ||
// since dynamic filter only supports left input ref in the output indices, we can safely use o2i mapping to convert the required columns. | ||
let o2i_mapping = self.core.o2i_col_mapping(); | ||
let left_input_columns = columns | ||
.iter() | ||
.map(|&col| o2i_mapping.try_map(col)) | ||
.collect::<Option<Vec<usize>>>()?; | ||
if let Some(better_left_plan) = self.left().try_better_locality(&left_input_columns) { | ||
return Some( | ||
self.clone_with_left_right(better_left_plan, self.right()) | ||
.into(), | ||
); | ||
} | ||
} | ||
None | ||
} | ||
} | ||
|
||
impl ToBatch for LogicalJoin { | ||
|
@@ -1754,23 +1774,19 @@ impl ToStream for LogicalJoin { | |
} | ||
|
||
fn try_better_locality(&self, columns: &[usize]) -> Option<PlanRef> { | ||
let mut ctx = ToStreamContext::new(false); | ||
// only pass through the locality information if it can be converted to dynamic filter | ||
if let Ok(Some(_)) = self.to_stream_dynamic_filter(self.on().clone(), &mut ctx) { | ||
// since dynamic filter only supports left input ref in the output indices, we can safely use o2i mapping to convert the required columns. | ||
let o2i_mapping = self.core.o2i_col_mapping(); | ||
let left_input_columns = columns | ||
.iter() | ||
.map(|&col| o2i_mapping.try_map(col)) | ||
.collect::<Option<Vec<usize>>>()?; | ||
if let Some(better_left_plan) = self.left().try_better_locality(&left_input_columns) { | ||
return Some( | ||
self.clone_with_left_right(better_left_plan, self.right()) | ||
.into(), | ||
); | ||
} | ||
if let Some(better_plan) = self.try_better_locality_inner(columns) { | ||
Some(better_plan) | ||
} else if self.ctx().session_ctx().config().enable_locality_backfill() { | ||
Some( | ||
LogicalLocalityProvider::new( | ||
self.clone_with_left_right(self.left(), self.right()).into(), | ||
columns.to_owned(), | ||
) | ||
.into(), | ||
) | ||
Comment on lines
+1780
to
+1786
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just realized that we generate a
instead of
Does it mean that we need to buffer unnecessary data that could have been filtered out by those stateless executors? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good insight. To resolve this issue, I think we should add the LocalityProvider to the requirement side instead of the provider side. |
||
} else { | ||
None | ||
} | ||
None | ||
} | ||
} | ||
|
||
|
Uh oh!
There was an error while loading. Please reload this page.