-
Notifications
You must be signed in to change notification settings - Fork 594
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
base: main
Are you sure you want to change the base?
Changes from all commits
605f235
501d374
3544c0e
d1a39a8
7c3f521
e3dbc73
b71eff9
9e0af8e
74336d6
96de9ba
3127678
49a48ad
bb7a29b
6b0b295
3c23aa3
3113463
b2e158e
fd308de
bf28307
369d718
3500061
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -39,6 +39,7 @@ use itertools::Itertools; | |
use parking_lot::Mutex; | ||
use rand::seq::SliceRandom; | ||
use rand::thread_rng; | ||
use risingwave_common::catalog::TableId; | ||
use risingwave_common::util::epoch::Epoch; | ||
use risingwave_hummock_sdk::compact_task::{CompactTask, ReportTask}; | ||
use risingwave_hummock_sdk::compaction_group::StateTableId; | ||
|
@@ -728,6 +729,46 @@ impl HummockManager { | |
} | ||
} | ||
|
||
// Filter out the table that has a primary key prefix watermark. | ||
let table_id_with_pk_prefix_watermark: HashSet<_> = self | ||
.metadata_manager | ||
.catalog_controller | ||
.get_table_by_ids( | ||
version | ||
.latest_version() | ||
.table_watermarks | ||
.keys() | ||
.map(|id| id.table_id() as _) | ||
.collect(), | ||
) | ||
.await | ||
.map_err(|e| Error::Internal(e.into()))? | ||
.into_iter() | ||
.filter_map(|table| { | ||
// pk prefix watermark. | ||
if table.clean_watermark_index_in_pk.is_none() | ||
|| table.clean_watermark_index_in_pk.unwrap() == 0 | ||
{ | ||
Some(TableId::from(table.get_id())) | ||
} else { | ||
None | ||
} | ||
}) | ||
.collect(); | ||
|
||
let table_watermarks = version | ||
.latest_version() | ||
.table_watermarks | ||
.iter() | ||
.filter_map(|(table_id, table_watermarks)| { | ||
if table_id_with_pk_prefix_watermark.contains(table_id) { | ||
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. We already have a 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. Also, if we filter out non pk prefix watermark here, how can compactor retrieve the non pk prefix watermark? Based on the logic here, it seems that we rely on the fact that non pk prefix watermark is present in the compact task. |
||
Some((*table_id, table_watermarks.clone())) | ||
} else { | ||
None | ||
} | ||
}) | ||
.collect(); | ||
|
||
while let Some(compact_task) = compact_status.get_compact_task( | ||
version | ||
.latest_version() | ||
|
@@ -742,7 +783,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; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -8,6 +8,7 @@ license = { workspace = true } | |
repository = { workspace = true } | ||
|
||
[dependencies] | ||
bincode = { version = "=2.0.0-rc.3", features = ["serde"] } | ||
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. ditto. See comments above. |
||
bytes = "1" | ||
hex = "0.4" | ||
itertools = { workspace = true } | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -12,7 +12,7 @@ | |
// See the License for the specific language governing permissions and | ||
// limitations under the License. | ||
|
||
use std::collections::{BTreeMap, HashMap}; | ||
use std::collections::{BTreeMap, HashMap, HashSet}; | ||
use std::mem::size_of; | ||
|
||
use itertools::Itertools; | ||
|
@@ -22,6 +22,7 @@ use risingwave_pb::hummock::{ | |
PbCompactTask, PbKeyRange, PbTableOption, PbTableSchema, PbTableStats, PbValidationTask, | ||
}; | ||
|
||
use crate::compaction_group::StateTableId; | ||
use crate::key_range::KeyRange; | ||
use crate::level::InputLevel; | ||
use crate::sstable_info::SstableInfo; | ||
|
@@ -114,6 +115,48 @@ impl CompactTask { | |
} | ||
} | ||
|
||
impl CompactTask { | ||
// The compact task may need to reclaim key with TTL | ||
pub fn is_contains_ttl(&self) -> bool { | ||
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. rename: |
||
self.table_options | ||
.iter() | ||
.any(|(_, table_option)| table_option.retention_seconds.is_some_and(|ttl| ttl > 0)) | ||
} | ||
|
||
// The compact task may need to reclaim key with range tombstone | ||
pub fn is_contains_range_tombstone(&self) -> bool { | ||
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. rename: |
||
self.input_ssts | ||
.iter() | ||
.flat_map(|level| level.table_infos.iter()) | ||
.any(|sst| sst.range_tombstone_count > 0) | ||
} | ||
|
||
// The compact task may need to reclaim key with split sst | ||
pub fn is_contains_split_sst(&self) -> bool { | ||
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. rename: |
||
self.input_ssts | ||
.iter() | ||
.flat_map(|level| level.table_infos.iter()) | ||
.any(|sst| sst.sst_id != sst.object_id) | ||
} | ||
|
||
pub fn get_table_ids_from_input_ssts(&self) -> impl Iterator<Item = StateTableId> { | ||
self.input_ssts | ||
.iter() | ||
.flat_map(|level| level.table_infos.iter()) | ||
.flat_map(|sst| sst.table_ids.clone()) | ||
.sorted() | ||
.unique() | ||
} | ||
|
||
// filter the table-id that in existing_table_ids with the table-id in compact-task | ||
pub fn build_compact_table_ids(&self) -> Vec<StateTableId> { | ||
let existing_table_ids: HashSet<u32> = HashSet::from_iter(self.existing_table_ids.clone()); | ||
self.get_table_ids_from_input_ssts() | ||
.filter(|table_id| existing_table_ids.contains(table_id)) | ||
.collect() | ||
} | ||
} | ||
|
||
impl From<PbCompactTask> for CompactTask { | ||
#[expect(deprecated)] | ||
fn from(pb_compact_task: PbCompactTask) -> Self { | ||
|
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.
Does this assume the pk prefix watermark can only be a single column? Do we have a sanity check somewhere to make sure this assumption holds?