Skip to content
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

fix(over window): fix over window range cache shortcut return #14311

Merged
merged 3 commits into from
Jan 3, 2024
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: 14 additions & 16 deletions e2e_test/streaming/over_window/main.slt
Original file line number Diff line number Diff line change
Expand Up @@ -5,22 +5,20 @@ set rw_streaming_over_window_cache_policy = full;

include ./generated/main.slt.part

# TODO(rc): The following tests are temporarily commented out because of recovery test failure.

#statement ok
#set rw_streaming_over_window_cache_policy = recent;
#
#include ./generated/main.slt.part
#
#statement ok
#set rw_streaming_over_window_cache_policy = recent_first_n;
#
#include ./generated/main.slt.part
#
#statement ok
#set rw_streaming_over_window_cache_policy = recent_last_n;
#
#include ./generated/main.slt.part
statement ok
set rw_streaming_over_window_cache_policy = recent;

include ./generated/main.slt.part

statement ok
set rw_streaming_over_window_cache_policy = recent_first_n;

include ./generated/main.slt.part

statement ok
set rw_streaming_over_window_cache_policy = recent_last_n;

include ./generated/main.slt.part

statement ok
set rw_streaming_over_window_cache_policy = default;
1 change: 1 addition & 0 deletions src/stream/src/executor/over_window/general.rs
Original file line number Diff line number Diff line change
Expand Up @@ -657,6 +657,7 @@ impl<S: StateStore> OverWindowExecutor<S> {
this.state_table.update_vnode_bitmap(vnode_bitmap);
if cache_may_stale {
vars.cached_partitions.clear();
vars.recently_accessed_ranges.clear();
}
}

Expand Down
18 changes: 8 additions & 10 deletions src/stream/src/executor/over_window/over_partition.rs
Original file line number Diff line number Diff line change
Expand Up @@ -506,10 +506,9 @@ impl<'a, S: StateStore> OverPartition<'a, S> {
.await;
}

// get the first and last keys again, now we are guaranteed to have at least a normal key
let cache_real_first_key = self.cache_real_first_key().unwrap();
let cache_real_last_key = self.cache_real_last_key().unwrap();

let cache_real_first_key = self
.cache_real_first_key()
.expect("cache real len is not 0");
if self.cache_left_is_sentinel() && *range.start() < cache_real_first_key {
// extend leftward only if there's smallest sentinel
let table_sub_range = (
Expand All @@ -524,11 +523,11 @@ impl<'a, S: StateStore> OverPartition<'a, S> {
table_sub_range=?table_sub_range,
"loading the left half of given range"
);
return self
.extend_cache_by_range_inner(table, table_sub_range)
.await;
self.extend_cache_by_range_inner(table, table_sub_range)
.await?;
}

let cache_real_last_key = self.cache_real_last_key().expect("cache real len is not 0");
if self.cache_right_is_sentinel() && *range.end() > cache_real_last_key {
// extend rightward only if there's largest sentinel
let table_sub_range = (
Expand All @@ -543,9 +542,8 @@ impl<'a, S: StateStore> OverPartition<'a, S> {
table_sub_range=?table_sub_range,
"loading the right half of given range"
);
return self
.extend_cache_by_range_inner(table, table_sub_range)
.await;
self.extend_cache_by_range_inner(table, table_sub_range)
.await?;
}

// TODO(rc): Uncomment the following to enable prefetching rows before the start of the
Expand Down
Loading