-
Notifications
You must be signed in to change notification settings - Fork 603
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
feat(storage): support non_pk_prefix_watermark state cleaning #19889
Changes from 25 commits
605f235
501d374
3544c0e
d1a39a8
7c3f521
e3dbc73
b71eff9
9e0af8e
74336d6
96de9ba
3127678
49a48ad
bb7a29b
6b0b295
3c23aa3
3113463
b2e158e
fd308de
bf28307
369d718
3500061
ef4c752
5ed4920
8130c61
7e8f6cd
ffded48
492b689
278912a
1166d45
7ccf804
4b3f518
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 | ||
---|---|---|---|---|
|
@@ -48,6 +48,7 @@ use risingwave_hummock_sdk::sstable_info::SstableInfo; | |||
use risingwave_hummock_sdk::table_stats::{ | ||||
add_prost_table_stats_map, purge_prost_table_stats, PbTableStatsMap, | ||||
}; | ||||
use risingwave_hummock_sdk::table_watermark::WatermarkSerdeType; | ||||
use risingwave_hummock_sdk::version::{GroupDelta, IntraLevelDelta}; | ||||
use risingwave_hummock_sdk::{ | ||||
compact_task_to_string, statistics_compact_task, CompactionGroupId, HummockCompactionTaskId, | ||||
|
@@ -728,6 +729,22 @@ impl HummockManager { | |||
} | ||||
} | ||||
|
||||
let table_watermarks = version | ||||
.latest_version() | ||||
.table_watermarks | ||||
.iter() | ||||
.filter_map(|(table_id, table_watermarks)| { | ||||
if matches!( | ||||
table_watermarks.watermark_type, | ||||
WatermarkSerdeType::PkPrefix, | ||||
) { | ||||
Some((*table_id, table_watermarks.clone())) | ||||
} else { | ||||
None | ||||
} | ||||
}) | ||||
.collect(); | ||||
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. Actually why don't we do the filtering inside the picker instead like in here if the watermark type is part of TableWatermarks:
We can avoid cloning the table watermark, which can be large given that it stores bytes from user data, with no harm. 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. Any ideas on this comment? @Li0k |
||||
|
||||
while let Some(compact_task) = compact_status.get_compact_task( | ||||
version | ||||
.latest_version() | ||||
|
@@ -742,7 +759,7 @@ impl HummockManager { | |||
selector, | ||||
&table_id_to_option, | ||||
developer_config.clone(), | ||||
&version.latest_version().table_watermarks, | ||||
&table_watermarks, | ||||
&version.latest_version().state_table_info, | ||||
) { | ||||
let target_level_id = compact_task.input.target_level as u32; | ||||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
should we assert
idx == 1
here?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, row can be any length and index can be a generic function.