diff --git a/src/ctl/src/cmd_impl/hummock/list_version.rs b/src/ctl/src/cmd_impl/hummock/list_version.rs index ac8ba6982316..0df76aacc9be 100644 --- a/src/ctl/src/cmd_impl/hummock/list_version.rs +++ b/src/ctl/src/cmd_impl/hummock/list_version.rs @@ -33,7 +33,7 @@ pub async fn list_version( { let l0 = &mut levels.l0; for sub_level in &mut l0.sub_levels { - for t in &mut sub_level.table_infos { + for t in &mut sub_level.sstable_infos { t.remove_key_range(); } } @@ -41,7 +41,7 @@ pub async fn list_version( // l1 ~ lmax for level in &mut levels.levels { - for t in &mut level.table_infos { + for t in &mut level.sstable_infos { t.remove_key_range(); } } @@ -65,7 +65,7 @@ pub async fn list_version( "sub_level_id {} type {} sst_num {} size {}", sub_level.sub_level_id, sub_level.level_type.as_str_name(), - sub_level.table_infos.len(), + sub_level.sstable_infos.len(), sub_level.total_file_size ) } @@ -76,7 +76,7 @@ pub async fn list_version( "level_idx {} type {} sst_num {} size {}", level.level_idx, level.level_type.as_str_name(), - level.table_infos.len(), + level.sstable_infos.len(), level.total_file_size ) } diff --git a/src/ctl/src/cmd_impl/hummock/sst_dump.rs b/src/ctl/src/cmd_impl/hummock/sst_dump.rs index 51b776ad1b2c..b347a0b58b7e 100644 --- a/src/ctl/src/cmd_impl/hummock/sst_dump.rs +++ b/src/ctl/src/cmd_impl/hummock/sst_dump.rs @@ -82,7 +82,7 @@ pub async fn sst_dump(context: &CtlContext, args: SstDumpArgs) -> anyhow::Result let version = hummock.inner().get_pinned_version().version().clone(); let sstable_store = hummock.sstable_store(); for level in version.get_combined_levels() { - for sstable_info in &level.table_infos { + for sstable_info in &level.sstable_infos { if let Some(object_id) = &args.object_id { if *object_id == sstable_info.object_id { print_level(level, sstable_info); diff --git a/src/ctl/src/cmd_impl/hummock/validate_version.rs b/src/ctl/src/cmd_impl/hummock/validate_version.rs index 3537a645c30e..c760b0733498 100644 --- a/src/ctl/src/cmd_impl/hummock/validate_version.rs +++ b/src/ctl/src/cmd_impl/hummock/validate_version.rs @@ -101,7 +101,7 @@ async fn print_user_key_in_version( println!("print key {:?} in version {}", target_key, version.id); for cg in version.levels.values() { for level in cg.l0.sub_levels.iter().rev().chain(cg.levels.iter()) { - for sstable_info in &level.table_infos { + for sstable_info in &level.sstable_infos { let key_range = &sstable_info.key_range; let left_user_key = FullKey::decode(&key_range.left); let right_user_key = FullKey::decode(&key_range.right); diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs index 7393c96ad426..61525c709b4e 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs @@ -87,7 +87,7 @@ fn remove_key_range_from_version(mut version: HummockVersion) -> HummockVersion // Because key range is too verbose for manual analysis, just don't expose it. for cg in version.levels.values_mut() { for level in cg.levels.iter_mut().chain(cg.l0.sub_levels.iter_mut()) { - for sst in &mut level.table_infos { + for sst in &mut level.sstable_infos { sst.remove_key_range(); } } @@ -111,7 +111,7 @@ fn version_to_sstable_rows(version: HummockVersion) -> Vec { let mut sstables = vec![]; for cg in version.levels.into_values() { for level in cg.levels.into_iter().chain(cg.l0.sub_levels) { - for sst in level.table_infos { + for sst in level.sstable_infos { let key_range = sst.key_range; sstables.push(RwHummockSstable { sstable_id: sst.sst_id as _, diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index 2ebbab619a79..fb39f40e2d1e 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -163,7 +163,7 @@ impl CompactStatus { } if task.input_ssts[1].level_idx == task.target_level - && task.input_ssts[1].table_infos.is_empty() + && task.input_ssts[1].sstable_infos.is_empty() { return true; } @@ -178,7 +178,7 @@ impl CompactStatus { } let exist_table_ids = HashSet::::from_iter(task.existing_table_ids.clone()); task.input_ssts.iter().all(|level| { - level.table_infos.iter().all(|sst| { + level.sstable_infos.iter().all(|sst| { sst.table_ids .iter() .all(|table_id| !exist_table_ids.contains(table_id)) diff --git a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs index 374cd999edd8..e0b6464840c8 100644 --- a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs @@ -52,7 +52,7 @@ impl CompactionPicker for LevelCompactionPicker { return None; } if l0.sub_levels[0].level_type != LevelType::Nonoverlapping - && l0.sub_levels[0].table_infos.len() > 1 + && l0.sub_levels[0].sstable_infos.len() > 1 { stats.skip_by_overlapping += 1; return None; @@ -145,8 +145,8 @@ impl LevelCompactionPicker { ); trivial_move_picker.pick_trivial_move_task( - &l0.sub_levels[0].table_infos, - &target_level.table_infos, + &l0.sub_levels[0].sstable_infos, + &target_level.sstable_infos, level_handlers, stats, ) @@ -208,7 +208,7 @@ impl LevelCompactionPicker { .collect_vec(); let target_level_ssts = overlap_strategy - .check_base_level_overlap(&l0_select_tables, &target_level.table_infos); + .check_base_level_overlap(&l0_select_tables, &target_level.sstable_infos); let mut target_level_size = 0; let mut pending_compact = false; @@ -243,7 +243,7 @@ impl LevelCompactionPicker { .map(|table_infos| InputLevel { level_idx: 0, level_type: LevelType::Nonoverlapping, - table_infos, + sstable_infos: table_infos, }) .collect_vec(); select_level_inputs.reverse(); @@ -251,7 +251,7 @@ impl LevelCompactionPicker { select_level_inputs.push(InputLevel { level_idx: target_level.level_idx, level_type: target_level.level_type, - table_infos: target_level_files, + sstable_infos: target_level_files, }); let result = CompactionInput { @@ -345,9 +345,9 @@ pub mod tests { let ret = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); - assert_eq!(ret.input_levels[0].table_infos.len(), 1); - assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 4); - assert_eq!(ret.input_levels[1].table_infos[0].sst_id, 1); + assert_eq!(ret.input_levels[0].sstable_infos.len(), 1); + assert_eq!(ret.input_levels[0].sstable_infos[0].sst_id, 4); + assert_eq!(ret.input_levels[1].sstable_infos[0].sst_id, 1); ret.add_pending_task(0, &mut levels_handler); { @@ -357,15 +357,15 @@ pub mod tests { .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); - assert_eq!(ret2.input_levels[0].table_infos.len(), 1); - assert_eq!(ret2.input_levels[0].table_infos[0].sst_id, 6); - assert_eq!(ret2.input_levels[1].table_infos[0].sst_id, 5); + assert_eq!(ret2.input_levels[0].sstable_infos.len(), 1); + assert_eq!(ret2.input_levels[0].sstable_infos[0].sst_id, 6); + assert_eq!(ret2.input_levels[1].sstable_infos[0].sst_id, 5); } levels.l0.sub_levels[0] - .table_infos + .sstable_infos .retain(|table| table.sst_id != 4); - levels.l0.total_file_size -= ret.input_levels[0].table_infos[0].file_size; + levels.l0.total_file_size -= ret.input_levels[0].sstable_infos[0].file_size; levels_handler[0].remove_task(0); levels_handler[1].remove_task(0); @@ -374,11 +374,11 @@ pub mod tests { .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); assert_eq!(ret.input_levels.len(), 3); - assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 6); - assert_eq!(ret.input_levels[1].table_infos[0].sst_id, 5); - assert_eq!(ret.input_levels[2].table_infos.len(), 2); - assert_eq!(ret.input_levels[2].table_infos[0].sst_id, 3); - assert_eq!(ret.input_levels[2].table_infos[1].sst_id, 2); + assert_eq!(ret.input_levels[0].sstable_infos[0].sst_id, 6); + assert_eq!(ret.input_levels[1].sstable_infos[0].sst_id, 5); + assert_eq!(ret.input_levels[2].sstable_infos.len(), 2); + assert_eq!(ret.input_levels[2].sstable_infos[0].sst_id, 3); + assert_eq!(ret.input_levels[2].sstable_infos[1].sst_id, 2); ret.add_pending_task(1, &mut levels_handler); let mut local_stats = LocalPickerStatistic::default(); @@ -395,9 +395,9 @@ pub mod tests { .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); assert_eq!(ret.input_levels.len(), 3); - assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 6); - assert_eq!(ret.input_levels[1].table_infos[0].sst_id, 5); - assert_eq!(ret.input_levels[2].table_infos.len(), 2); + assert_eq!(ret.input_levels[0].sstable_infos[0].sst_id, 6); + assert_eq!(ret.input_levels[1].sstable_infos[0].sst_id, 5); + assert_eq!(ret.input_levels[2].sstable_infos.len(), 2); } #[test] fn test_selecting_key_range_overlap() { @@ -415,7 +415,7 @@ pub mod tests { let levels = vec![Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(3, 1, 0, 50, 1), generate_table(4, 1, 150, 200, 1), generate_table(5, 1, 250, 300, 1), @@ -453,7 +453,7 @@ pub mod tests { assert_eq!(ret.input_levels.len(), 2); assert_eq!( ret.input_levels[0] - .table_infos + .sstable_infos .iter() .map(|t| t.sst_id) .collect_vec(), @@ -462,7 +462,7 @@ pub mod tests { assert_eq!( ret.input_levels[1] - .table_infos + .sstable_infos .iter() .map(|t| t.sst_id) .collect_vec(), @@ -478,7 +478,7 @@ pub mod tests { let levels = vec![Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![], + sstable_infos: vec![], total_file_size: 0, sub_level_id: 0, uncompressed_file_size: 0, @@ -542,7 +542,7 @@ pub mod tests { levels: vec![Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(1, 1, 100, 399, 2), generate_table(2, 1, 400, 699, 2), generate_table(3, 1, 700, 999, 2), @@ -566,7 +566,7 @@ pub mod tests { let mut levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; let mut local_stats = LocalPickerStatistic::default(); - levels_handler[0].add_pending_task(1, 4, &levels.l0.sub_levels[0].table_infos); + levels_handler[0].add_pending_task(1, 4, &levels.l0.sub_levels[0].sstable_infos); let ret = picker.pick_compaction(&levels, &levels_handler, &mut local_stats); // Skip this compaction because the write amplification is too large. assert!(ret.is_none()); @@ -610,7 +610,7 @@ pub mod tests { let ret = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); - assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 7); + assert_eq!(ret.input_levels[0].sstable_infos[0].sst_id, 7); assert_eq!( 3, ret.input_levels.iter().filter(|l| l.level_idx == 0).count() @@ -620,7 +620,7 @@ pub mod tests { ret.input_levels .iter() .filter(|l| l.level_idx == 0) - .map(|l| l.table_infos.len()) + .map(|l| l.sstable_infos.len()) .sum::() ); @@ -638,7 +638,7 @@ pub mod tests { let ret = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); - assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 6); + assert_eq!(ret.input_levels[0].sstable_infos[0].sst_id, 6); assert_eq!( 2, ret.input_levels.iter().filter(|l| l.level_idx == 0).count() @@ -648,7 +648,7 @@ pub mod tests { ret.input_levels .iter() .filter(|l| l.level_idx == 0) - .map(|l| l.table_infos.len()) + .map(|l| l.sstable_infos.len()) .sum::() ); } @@ -679,7 +679,7 @@ pub mod tests { input_levels: vec![InputLevel { level_idx: 0, level_type: pending_level.level_type, - table_infos: pending_level.table_infos.clone(), + sstable_infos: pending_level.sstable_infos.clone(), }], target_level: 1, target_sub_level_id: pending_level.sub_level_id, @@ -753,14 +753,14 @@ pub mod tests { .unwrap(); // 1. trivial_move assert_eq!(2, ret.input_levels.len()); - assert!(ret.input_levels[1].table_infos.is_empty()); - assert_eq!(5, ret.input_levels[0].table_infos[0].sst_id); + assert!(ret.input_levels[1].sstable_infos.is_empty()); + assert_eq!(5, ret.input_levels[0].sstable_infos[0].sst_id); ret.add_pending_task(0, &mut levels_handler); let ret = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); assert_eq!(3, ret.input_levels.len()); - assert_eq!(6, ret.input_levels[0].table_infos[0].sst_id); + assert_eq!(6, ret.input_levels[0].sstable_infos[0].sst_id); } } diff --git a/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs b/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs index acf803d1ce57..8448031d992e 100644 --- a/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs +++ b/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs @@ -155,7 +155,7 @@ impl CompactionTaskValidationRule for IntraCompactionTaskValidationRule { let mut max_level_size = 0; for select_level in &input.input_levels { let level_select_size = select_level - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_size) .sum::(); diff --git a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs index 644b70485d0f..5b89d1ae34f1 100644 --- a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs @@ -205,7 +205,7 @@ impl IntraCompactionPicker { select_level_inputs.push(InputLevel { level_idx: 0, level_type: LevelType::Nonoverlapping, - table_infos: level_select_sst, + sstable_infos: level_select_sst, }); select_input_size += input.total_file_size; @@ -270,8 +270,8 @@ impl IntraCompactionPicker { let trivial_move_picker = TrivialMovePicker::new(0, 0, overlap_strategy.clone(), 0); let select_sst = trivial_move_picker.pick_trivial_move_sst( - &l0.sub_levels[idx + 1].table_infos, - &level.table_infos, + &l0.sub_levels[idx + 1].sstable_infos, + &level.sstable_infos, level_handlers, stats, ); @@ -288,7 +288,7 @@ impl IntraCompactionPicker { overlap.update(&select_sst); assert!(overlap - .check_multiple_overlap(&l0.sub_levels[idx].table_infos) + .check_multiple_overlap(&l0.sub_levels[idx].sstable_infos) .is_empty()); let select_input_size = select_sst.sst_size; @@ -296,12 +296,12 @@ impl IntraCompactionPicker { InputLevel { level_idx: 0, level_type: LevelType::Nonoverlapping, - table_infos: vec![select_sst], + sstable_infos: vec![select_sst], }, InputLevel { level_idx: 0, level_type: LevelType::Nonoverlapping, - table_infos: vec![], + sstable_infos: vec![], }, ]; return Some(CompactionInput { @@ -376,12 +376,12 @@ impl WholeLevelCompactionPicker { } select_input_size += next_level.total_file_size; - total_file_count += next_level.table_infos.len() as u64; + total_file_count += next_level.sstable_infos.len() as u64; select_level_inputs.push(InputLevel { level_idx: 0, level_type: next_level.level_type, - table_infos: next_level.table_infos.clone(), + sstable_infos: next_level.sstable_infos.clone(), }); } if select_level_inputs.len() > 1 { @@ -445,7 +445,7 @@ pub mod tests { let levels = vec![Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![], + sstable_infos: vec![], ..Default::default() }]; let mut levels = Levels { @@ -492,7 +492,7 @@ pub mod tests { levels: vec![Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![generate_table(3, 1, 200, 300, 2)], + sstable_infos: vec![generate_table(3, 1, 200, 300, 2)], ..Default::default() }], l0: generate_l0_nonoverlapping_sublevels(vec![ @@ -539,7 +539,7 @@ pub mod tests { ..Default::default() }; let mut levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; - levels_handler[1].add_pending_task(100, 1, &levels.levels[0].table_infos); + levels_handler[1].add_pending_task(100, 1, &levels.levels[0].sstable_infos); let config = Arc::new( CompactionConfigBuilder::new() .level0_sub_level_compact_level_count(1) @@ -558,7 +558,7 @@ pub mod tests { assert_eq!( ret.input_levels .iter() - .map(|i| i.table_infos.len()) + .map(|i| i.sstable_infos.len()) .sum::(), 3 ); @@ -587,7 +587,7 @@ pub mod tests { ..Default::default() }; let mut levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; - levels_handler[1].add_pending_task(100, 1, &levels.levels[0].table_infos); + levels_handler[1].add_pending_task(100, 1, &levels.levels[0].sstable_infos); let config = Arc::new( CompactionConfigBuilder::new() .level0_sub_level_compact_level_count(1) @@ -606,14 +606,14 @@ pub mod tests { assert_eq!( ret.input_levels .iter() - .map(|i| i.table_infos.len()) + .map(|i| i.sstable_infos.len()) .sum::(), 3 ); - assert_eq!(4, ret.input_levels[0].table_infos[0].sst_id); - assert_eq!(3, ret.input_levels[1].table_infos[0].sst_id); - assert_eq!(1, ret.input_levels[2].table_infos[0].sst_id); + assert_eq!(4, ret.input_levels[0].sstable_infos[0].sst_id); + assert_eq!(3, ret.input_levels[1].sstable_infos[0].sst_id); + assert_eq!(1, ret.input_levels[2].sstable_infos[0].sst_id); // will pick sst [2, 6] let ret2 = picker @@ -623,13 +623,13 @@ pub mod tests { assert_eq!( ret2.input_levels .iter() - .map(|i| i.table_infos.len()) + .map(|i| i.sstable_infos.len()) .sum::(), 2 ); - assert_eq!(6, ret2.input_levels[0].table_infos[0].sst_id); - assert_eq!(2, ret2.input_levels[1].table_infos[0].sst_id); + assert_eq!(6, ret2.input_levels[0].sstable_infos[0].sst_id); + assert_eq!(2, ret2.input_levels[1].sstable_infos[0].sst_id); } { @@ -658,7 +658,7 @@ pub mod tests { ..Default::default() }; let mut levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; - levels_handler[1].add_pending_task(100, 1, &levels.levels[0].table_infos); + levels_handler[1].add_pending_task(100, 1, &levels.levels[0].sstable_infos); let config = Arc::new( CompactionConfigBuilder::new() .level0_sub_level_compact_level_count(1) @@ -677,14 +677,14 @@ pub mod tests { assert_eq!( ret.input_levels .iter() - .map(|i| i.table_infos.len()) + .map(|i| i.sstable_infos.len()) .sum::(), 3 ); - assert_eq!(11, ret.input_levels[0].table_infos[0].sst_id); - assert_eq!(9, ret.input_levels[1].table_infos[0].sst_id); - assert_eq!(7, ret.input_levels[2].table_infos[0].sst_id); + assert_eq!(11, ret.input_levels[0].sstable_infos[0].sst_id); + assert_eq!(9, ret.input_levels[1].sstable_infos[0].sst_id); + assert_eq!(7, ret.input_levels[2].sstable_infos[0].sst_id); let ret2 = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) @@ -693,21 +693,21 @@ pub mod tests { assert_eq!( ret2.input_levels .iter() - .map(|i| i.table_infos.len()) + .map(|i| i.sstable_infos.len()) .sum::(), 3 ); - assert_eq!(5, ret2.input_levels[0].table_infos[0].sst_id); - assert_eq!(10, ret2.input_levels[1].table_infos[0].sst_id); - assert_eq!(2, ret2.input_levels[2].table_infos[0].sst_id); + assert_eq!(5, ret2.input_levels[0].sstable_infos[0].sst_id); + assert_eq!(10, ret2.input_levels[1].sstable_infos[0].sst_id); + assert_eq!(2, ret2.input_levels[2].sstable_infos[0].sst_id); } } fn is_l0_trivial_move(compaction_input: &CompactionInput) -> bool { compaction_input.input_levels.len() == 2 - && !compaction_input.input_levels[0].table_infos.is_empty() - && compaction_input.input_levels[1].table_infos.is_empty() + && !compaction_input.input_levels[0].sstable_infos.is_empty() + && compaction_input.input_levels[1].sstable_infos.is_empty() } #[test] @@ -733,7 +733,7 @@ pub mod tests { levels: vec![generate_level(1, vec![generate_table(100, 1, 0, 1000, 1)])], ..Default::default() }; - levels_handler[1].add_pending_task(100, 1, &levels.levels[0].table_infos); + levels_handler[1].add_pending_task(100, 1, &levels.levels[0].sstable_infos); let mut local_stats = LocalPickerStatistic::default(); let ret = picker.pick_compaction(&levels, &levels_handler, &mut local_stats); assert!(ret.is_none()); @@ -776,7 +776,7 @@ pub mod tests { .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); assert!(is_l0_trivial_move(&ret)); - assert_eq!(ret.input_levels[0].table_infos.len(), 1); + assert_eq!(ret.input_levels[0].sstable_infos.len(), 1); } #[test] fn test_pick_whole_level() { @@ -831,7 +831,7 @@ pub mod tests { let mut l0 = generate_l0_nonoverlapping_multi_sublevels(table_infos); // trivial-move l0.sub_levels[1] - .table_infos + .sstable_infos .push(generate_table(9999, 900000000, 0, 100, 1)); l0.sub_levels[0].total_file_size = 1; @@ -861,12 +861,12 @@ pub mod tests { let input = ret.as_ref().unwrap(); assert_eq!(input.input_levels.len(), 2); assert_ne!( - levels.l0.sub_levels[0].table_infos.len(), - input.input_levels[0].table_infos.len() + levels.l0.sub_levels[0].sstable_infos.len(), + input.input_levels[0].sstable_infos.len() ); assert_ne!( - levels.l0.sub_levels[1].table_infos.len(), - input.input_levels[1].table_infos.len() + levels.l0.sub_levels[1].sstable_infos.len(), + input.input_levels[1].sstable_infos.len() ); } } diff --git a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs index 597d18cce4fd..97da61424a7d 100644 --- a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs @@ -90,7 +90,7 @@ impl ManualCompactionPicker { input_levels.push(InputLevel { level_idx: 0, level_type: level.level_type, - table_infos: level.table_infos.clone(), + sstable_infos: level.sstable_infos.clone(), }); } if input_levels.is_empty() { @@ -112,7 +112,7 @@ impl ManualCompactionPicker { ) -> Option { assert!(self.option.level == 0 && self.target_level > 0); for l in 1..self.target_level { - assert!(levels.levels[l - 1].table_infos.is_empty()); + assert!(levels.levels[l - 1].sstable_infos.is_empty()); } let l0 = &levels.l0; let mut input_levels = vec![]; @@ -135,21 +135,21 @@ impl ManualCompactionPicker { } // Construct input. for idx in 0..=max_sub_level_idx { - for table in &l0.sub_levels[idx].table_infos { + for table in &l0.sub_levels[idx].sstable_infos { info.update(table); } input_levels.push(InputLevel { level_idx: 0, level_type: l0.sub_levels[idx].level_type, - table_infos: l0.sub_levels[idx].table_infos.clone(), + sstable_infos: l0.sub_levels[idx].sstable_infos.clone(), }) } let target_input_ssts_range = - info.check_multiple_overlap(&levels.levels[self.target_level - 1].table_infos); + info.check_multiple_overlap(&levels.levels[self.target_level - 1].sstable_infos); let target_input_ssts = if target_input_ssts_range.is_empty() { vec![] } else { - levels.levels[self.target_level - 1].table_infos[target_input_ssts_range].to_vec() + levels.levels[self.target_level - 1].sstable_infos[target_input_ssts_range].to_vec() }; if target_input_ssts .iter() @@ -164,7 +164,7 @@ impl ManualCompactionPicker { input_levels.push(InputLevel { level_idx: self.target_level as u32, level_type: LevelType::Nonoverlapping, - table_infos: target_input_ssts, + sstable_infos: target_input_ssts, }); Some(CompactionInput { @@ -186,21 +186,21 @@ impl ManualCompactionPicker { }; if self .overlap_strategy - .check_overlap_with_tables(&[tmp_sst_info], &level.table_infos) + .check_overlap_with_tables(&[tmp_sst_info], &level.sstable_infos) .is_empty() { return false; } if !hint_sst_ids.is_empty() && !level - .table_infos + .sstable_infos .iter() .any(|t| hint_sst_ids.contains(&t.sst_id)) { return false; } if !self.option.internal_table_id.is_empty() - && !level.table_infos.iter().any(|sst_info| { + && !level.sstable_infos.iter().any(|sst_info| { sst_info .table_ids .iter() @@ -243,7 +243,7 @@ impl CompactionPicker for ManualCompactionPicker { // We either include all `select_input_ssts` as input, or return None. let mut select_input_ssts: Vec = levels .get_level(self.option.level) - .table_infos + .sstable_infos .iter() .filter(|sst_info| hint_sst_ids.is_empty() || hint_sst_ids.contains(&sst_info.sst_id)) .filter(|sst_info| range_overlap_info.check_overlap(sst_info)) @@ -269,22 +269,22 @@ impl CompactionPicker for ManualCompactionPicker { // For intra level compaction, input SSTs must be consecutive. let (left, _) = levels .get_level(level) - .table_infos + .sstable_infos .iter() .find_position(|p| p.sst_id == select_input_ssts.first().unwrap().sst_id) .unwrap(); let (right, _) = levels .get_level(level) - .table_infos + .sstable_infos .iter() .find_position(|p| p.sst_id == select_input_ssts.last().unwrap().sst_id) .unwrap(); - select_input_ssts = levels.get_level(level).table_infos[left..=right].to_vec(); + select_input_ssts = levels.get_level(level).sstable_infos[left..=right].to_vec(); vec![] } else { self.overlap_strategy.check_base_level_overlap( &select_input_ssts, - &levels.get_level(target_level).table_infos, + &levels.get_level(target_level).sstable_infos, ) }; if select_input_ssts @@ -308,12 +308,12 @@ impl CompactionPicker for ManualCompactionPicker { InputLevel { level_idx: level as u32, level_type: levels.levels[level - 1].level_type, - table_infos: select_input_ssts, + sstable_infos: select_input_ssts, }, InputLevel { level_idx: target_level as u32, level_type: levels.levels[target_level - 1].level_type, - table_infos: target_input_ssts, + sstable_infos: target_input_ssts, }, ], target_level, @@ -379,7 +379,7 @@ pub mod tests { Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(0, 1, 0, 100, 1), generate_table(1, 1, 101, 200, 1), generate_table(2, 1, 222, 300, 1), @@ -389,7 +389,7 @@ pub mod tests { Level { level_idx: 2, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(4, 1, 0, 100, 1), generate_table(5, 1, 101, 150, 1), generate_table(6, 1, 151, 201, 1), @@ -434,8 +434,8 @@ pub mod tests { .unwrap(); result.add_pending_task(0, &mut levels_handler); - assert_eq!(2, result.input_levels[0].table_infos.len()); - assert_eq!(3, result.input_levels[1].table_infos.len()); + assert_eq!(2, result.input_levels[0].sstable_infos.len()); + assert_eq!(3, result.input_levels[1].sstable_infos.len()); } { @@ -455,15 +455,15 @@ pub mod tests { .unwrap(); result.add_pending_task(0, &mut levels_handler); - assert_eq!(3, result.input_levels[0].table_infos.len()); - assert_eq!(3, result.input_levels[1].table_infos.len()); + assert_eq!(3, result.input_levels[0].sstable_infos.len()); + assert_eq!(3, result.input_levels[1].sstable_infos.len()); } { clean_task_state(&mut levels_handler[1]); clean_task_state(&mut levels_handler[2]); - let level_table_info = &mut levels.levels[0].table_infos; + let level_table_info = &mut levels.levels[0].sstable_infos; let table_info_1 = &mut level_table_info[1]; table_info_1.table_ids.resize(2, 0); table_info_1.table_ids[0] = 1; @@ -488,8 +488,8 @@ pub mod tests { .unwrap(); result.add_pending_task(0, &mut levels_handler); - assert_eq!(1, result.input_levels[0].table_infos.len()); - assert_eq!(2, result.input_levels[1].table_infos.len()); + assert_eq!(1, result.input_levels[0].sstable_infos.len()); + assert_eq!(2, result.input_levels[1].sstable_infos.len()); } { @@ -497,7 +497,7 @@ pub mod tests { clean_task_state(&mut levels_handler[2]); // include all table_info - let level_table_info = &mut levels.levels[0].table_infos; + let level_table_info = &mut levels.levels[0].sstable_infos; for table_info in level_table_info { table_info.table_ids.resize(2, 0); table_info.table_ids[0] = 1; @@ -527,8 +527,8 @@ pub mod tests { .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); - assert_eq!(1, result.input_levels[0].table_infos.len()); - assert_eq!(2, result.input_levels[1].table_infos.len()); + assert_eq!(1, result.input_levels[0].sstable_infos.len()); + assert_eq!(2, result.input_levels[1].sstable_infos.len()); } } @@ -554,7 +554,7 @@ pub mod tests { Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(3, 1, 0, 100, 1), generate_table(4, 2, 2000, 3000, 1), ], @@ -563,7 +563,7 @@ pub mod tests { Level { level_idx: 2, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(1, 1, 0, 100, 1), generate_table(2, 2, 2000, 3000, 1), ], @@ -574,7 +574,7 @@ pub mod tests { assert_eq!(levels.len(), 2); for iter in [l0.sub_levels.iter_mut(), levels.iter_mut()] { for (idx, l) in iter.enumerate() { - for t in &mut l.table_infos { + for t in &mut l.sstable_infos { t.table_ids.clear(); if idx == 0 { t.table_ids.push(((t.sst_id % 2) + 1) as _); @@ -603,7 +603,7 @@ pub mod tests { let levels = vec![Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(1, 1, 0, 100, 1), generate_table(2, 2, 100, 200, 1), generate_table(3, 2, 200, 300, 1), @@ -627,7 +627,7 @@ pub mod tests { let levels = vec![Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![], + sstable_infos: vec![], total_file_size: 0, sub_level_id: 0, uncompressed_file_size: 0, @@ -699,7 +699,7 @@ pub mod tests { for (l, e) in expected.iter().enumerate().take(3) { assert_eq!( result.input_levels[l] - .table_infos + .sstable_infos .iter() .map(|s| s.sst_id) .collect_vec(), @@ -707,8 +707,8 @@ pub mod tests { ); } assert_eq!( - result.input_levels[3].table_infos, - vec![levels.levels[0].table_infos[0].clone()] + result.input_levels[3].sstable_infos, + vec![levels.levels[0].sstable_infos[0].clone()] ); // pick_l0_to_base_level, filtered by key_range @@ -735,7 +735,7 @@ pub mod tests { for (l, e) in expected.iter().enumerate().take(2) { assert_eq!( result.input_levels[l] - .table_infos + .sstable_infos .iter() .map(|s| s.sst_id) .collect_vec(), @@ -743,8 +743,8 @@ pub mod tests { ); } assert_eq!( - result.input_levels[2].table_infos, - vec![levels.levels[0].table_infos[0].clone()] + result.input_levels[2].sstable_infos, + vec![levels.levels[0].sstable_infos[0].clone()] ); } @@ -787,7 +787,7 @@ pub mod tests { for (i, e) in expected.iter().enumerate().take(result.input_levels.len()) { assert_eq!( result.input_levels[i] - .table_infos + .sstable_infos .iter() .map(|s| s.sst_id) .collect_vec(), @@ -854,14 +854,14 @@ pub mod tests { .input_levels .iter() .take(3) - .flat_map(|s| s.table_infos.clone()) + .flat_map(|s| s.sstable_infos.clone()) .map(|s| s.sst_id) .collect_vec(), vec![9, 10, 7, 8, 5, 6] ); assert_eq!( result.input_levels[3] - .table_infos + .sstable_infos .iter() .map(|s| s.sst_id) .collect_vec(), @@ -896,14 +896,14 @@ pub mod tests { .input_levels .iter() .take(3) - .flat_map(|s| s.table_infos.clone()) + .flat_map(|s| s.sstable_infos.clone()) .map(|s| s.sst_id) .collect_vec(), vec![9, 10, 7, 8, 5, 6] ); assert_eq!( result.input_levels[3] - .table_infos + .sstable_infos .iter() .map(|s| s.sst_id) .collect_vec(), @@ -942,14 +942,14 @@ pub mod tests { .input_levels .iter() .take(1) - .flat_map(|s| s.table_infos.clone()) + .flat_map(|s| s.sstable_infos.clone()) .map(|s| s.sst_id) .collect_vec(), vec![5, 6] ); assert_eq!( result.input_levels[1] - .table_infos + .sstable_infos .iter() .map(|s| s.sst_id) .collect_vec(), @@ -1044,7 +1044,7 @@ pub mod tests { { assert_eq!( result.input_levels[l] - .table_infos + .sstable_infos .iter() .map(|s| s.sst_id) .collect_vec(), @@ -1087,7 +1087,7 @@ pub mod tests { for (i, e) in expected.iter().enumerate().take(result.input_levels.len()) { assert_eq!( result.input_levels[i] - .table_infos + .sstable_infos .iter() .map(|s| s.sst_id) .collect_vec(), @@ -1137,7 +1137,7 @@ pub mod tests { for (i, e) in expected.iter().enumerate().take(result.input_levels.len()) { assert_eq!( result.input_levels[i] - .table_infos + .sstable_infos .iter() .map(|s| s.sst_id) .collect_vec(), @@ -1163,7 +1163,7 @@ pub mod tests { Level { level_idx: 4, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(2, 1, 0, 100, 1), generate_table(3, 1, 101, 200, 1), generate_table(4, 1, 222, 300, 1), @@ -1280,7 +1280,7 @@ pub mod tests { Level { level_idx: 4, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(2, 1, 0, 100, 1), generate_table(3, 1, 101, 200, 1), generate_table(4, 1, 222, 300, 1), @@ -1332,9 +1332,9 @@ pub mod tests { assert_compaction_task(&task, &levels_handler); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 3); - assert_eq!(task.input.input_levels[0].table_infos.len(), 2); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 2); assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 2); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 2); assert_eq!(task.input.target_level, 4); } @@ -1376,9 +1376,9 @@ pub mod tests { assert_compaction_task(&task, &levels_handler); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 4); - assert_eq!(task.input.input_levels[0].table_infos.len(), 6); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 6); assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, diff --git a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs index 4d7b5ac1e72f..64ad093c29f3 100644 --- a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs @@ -140,8 +140,8 @@ impl CompactionPicker for MinOverlappingPicker { ) -> Option { assert!(self.level > 0); let (select_input_ssts, target_input_ssts) = self.pick_tables( - &levels.get_level(self.level).table_infos, - &levels.get_level(self.target_level).table_infos, + &levels.get_level(self.level).sstable_infos, + &levels.get_level(self.target_level).sstable_infos, level_handlers, ); if select_input_ssts.is_empty() { @@ -156,12 +156,12 @@ impl CompactionPicker for MinOverlappingPicker { InputLevel { level_idx: self.level as u32, level_type: LevelType::Nonoverlapping, - table_infos: select_input_ssts, + sstable_infos: select_input_ssts, }, InputLevel { level_idx: self.target_level as u32, level_type: LevelType::Nonoverlapping, - table_infos: target_input_ssts, + sstable_infos: target_input_ssts, }, ], target_level: self.target_level, @@ -272,10 +272,10 @@ impl NonOverlapSubLevelPicker { basic_overlap_info.update(sst); let mut overlap_files_range = - basic_overlap_info.check_multiple_include(&target_level.table_infos); + basic_overlap_info.check_multiple_include(&target_level.sstable_infos); if overlap_files_range.is_empty() { overlap_files_range = - basic_overlap_info.check_multiple_overlap(&target_level.table_infos); + basic_overlap_info.check_multiple_overlap(&target_level.sstable_infos); } if overlap_files_range.is_empty() { @@ -289,7 +289,7 @@ impl NonOverlapSubLevelPicker { let mut select_level_count = 0; for reverse_index in (0..=target_index).rev() { - let target_tables = &levels[reverse_index].table_infos; + let target_tables = &levels[reverse_index].sstable_infos; overlap_files_range = if target_index == reverse_index { overlap_files_range @@ -334,7 +334,7 @@ impl NonOverlapSubLevelPicker { if !pick_levels_range.is_empty() { for (reverse_index, sst_range) in pick_levels_range { - let level_ssts = &levels[reverse_index].table_infos; + let level_ssts = &levels[reverse_index].sstable_infos; ret.sstable_infos[reverse_index] = level_ssts[sst_range].to_vec(); ret.total_file_count += ret.sstable_infos[reverse_index].len(); ret.total_file_size += ret.sstable_infos[reverse_index] @@ -400,7 +400,7 @@ impl NonOverlapSubLevelPicker { } let mut scores = vec![]; - for sst in &l0[0].table_infos { + for sst in &l0[0].sstable_infos { if level_handler.is_pending_compact(&sst.sst_id) { continue; } @@ -482,9 +482,9 @@ impl NonOverlapSubLevelPicker { if let Some(overlap_info) = overlap_info.as_mut() { // skip the check if `overlap_info` is not initialized (i.e. the first non-empty level is not met) let level = levels.get(level_idx).unwrap(); - let overlap_sst_range = overlap_info.check_multiple_overlap(&level.table_infos); + let overlap_sst_range = overlap_info.check_multiple_overlap(&level.sstable_infos); if !overlap_sst_range.is_empty() { - let expected_sst_ids = level.table_infos[overlap_sst_range.clone()] + let expected_sst_ids = level.sstable_infos[overlap_sst_range.clone()] .iter() .map(|s| s.object_id) .collect_vec(); @@ -504,7 +504,7 @@ impl NonOverlapSubLevelPicker { // Print SstableInfo for `expected_sst_ids` let mut expected_sst_infos = String::new(); - level.table_infos[overlap_sst_range.clone()] + level.sstable_infos[overlap_sst_range.clone()] .iter() .for_each(|s| { append_sstable_info_to_string(&mut expected_sst_infos, s) @@ -564,7 +564,7 @@ pub mod tests { Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(0, 1, 0, 100, 1), generate_table(1, 1, 101, 200, 1), generate_table(2, 1, 222, 300, 1), @@ -574,7 +574,7 @@ pub mod tests { Level { level_idx: 2, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(4, 1, 0, 100, 1), generate_table(5, 1, 101, 150, 1), generate_table(6, 1, 151, 201, 1), @@ -603,9 +603,9 @@ pub mod tests { .unwrap(); assert_eq!(ret.input_levels[0].level_idx, 1); assert_eq!(ret.target_level, 2); - assert_eq!(ret.input_levels[0].table_infos.len(), 1); - assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 2); - assert_eq!(ret.input_levels[1].table_infos.len(), 0); + assert_eq!(ret.input_levels[0].sstable_infos.len(), 1); + assert_eq!(ret.input_levels[0].sstable_infos[0].sst_id, 2); + assert_eq!(ret.input_levels[1].sstable_infos.len(), 0); ret.add_pending_task(0, &mut level_handlers); let ret = picker @@ -613,19 +613,19 @@ pub mod tests { .unwrap(); assert_eq!(ret.input_levels[0].level_idx, 1); assert_eq!(ret.target_level, 2); - assert_eq!(ret.input_levels[0].table_infos.len(), 1); - assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 0); - assert_eq!(ret.input_levels[1].table_infos.len(), 1); - assert_eq!(ret.input_levels[1].table_infos[0].sst_id, 4); + assert_eq!(ret.input_levels[0].sstable_infos.len(), 1); + assert_eq!(ret.input_levels[0].sstable_infos[0].sst_id, 0); + assert_eq!(ret.input_levels[1].sstable_infos.len(), 1); + assert_eq!(ret.input_levels[1].sstable_infos[0].sst_id, 4); ret.add_pending_task(1, &mut level_handlers); let ret = picker .pick_compaction(&levels, &level_handlers, &mut local_stats) .unwrap(); - assert_eq!(ret.input_levels[0].table_infos.len(), 1); - assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 1); - assert_eq!(ret.input_levels[1].table_infos.len(), 2); - assert_eq!(ret.input_levels[1].table_infos[0].sst_id, 5); + assert_eq!(ret.input_levels[0].sstable_infos.len(), 1); + assert_eq!(ret.input_levels[0].sstable_infos[0].sst_id, 1); + assert_eq!(ret.input_levels[1].sstable_infos.len(), 2); + assert_eq!(ret.input_levels[1].sstable_infos[0].sst_id, 5); } #[test] @@ -636,7 +636,7 @@ pub mod tests { Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(0, 1, 50, 99, 2), generate_table(1, 1, 100, 149, 2), generate_table(2, 1, 150, 249, 2), @@ -646,7 +646,7 @@ pub mod tests { Level { level_idx: 2, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(4, 1, 50, 199, 1), generate_table(5, 1, 200, 399, 1), ], @@ -676,12 +676,12 @@ pub mod tests { assert_eq!(ret.input_levels[0].level_idx, 1); assert_eq!(ret.input_levels[1].level_idx, 2); - assert_eq!(ret.input_levels[0].table_infos.len(), 2); - assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 0); - assert_eq!(ret.input_levels[0].table_infos[1].sst_id, 1); + assert_eq!(ret.input_levels[0].sstable_infos.len(), 2); + assert_eq!(ret.input_levels[0].sstable_infos[0].sst_id, 0); + assert_eq!(ret.input_levels[0].sstable_infos[1].sst_id, 1); - assert_eq!(ret.input_levels[1].table_infos.len(), 1); - assert_eq!(ret.input_levels[1].table_infos[0].sst_id, 4); + assert_eq!(ret.input_levels[1].sstable_infos.len(), 1); + assert_eq!(ret.input_levels[1].sstable_infos[0].sst_id, 4); } #[test] @@ -690,7 +690,7 @@ pub mod tests { Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(0, 1, 50, 99, 2), generate_table(1, 1, 100, 149, 2), generate_table(2, 1, 150, 249, 2), @@ -704,7 +704,7 @@ pub mod tests { Level { level_idx: 2, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(4, 1, 50, 199, 1), generate_table(5, 1, 200, 249, 1), generate_table(9, 1, 250, 300, 2), @@ -717,7 +717,7 @@ pub mod tests { Level { level_idx: 3, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(11, 1, 250, 300, 2), generate_table(12, 1, 350, 400, 2), generate_table(13, 1, 450, 500, 2), @@ -728,7 +728,7 @@ pub mod tests { Level { level_idx: 4, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(14, 1, 250, 300, 2), generate_table(15, 1, 350, 400, 2), generate_table(16, 1, 450, 500, 2), @@ -796,7 +796,7 @@ pub mod tests { Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(0, 1, 50, 99, 2), generate_table(1, 1, 100, 149, 2), generate_table(2, 1, 150, 249, 2), @@ -810,7 +810,7 @@ pub mod tests { Level { level_idx: 2, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(4, 1, 50, 99, 1), generate_table(5, 1, 150, 200, 1), generate_table(9, 1, 250, 300, 2), @@ -823,7 +823,7 @@ pub mod tests { Level { level_idx: 3, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(11, 1, 250, 300, 2), generate_table(12, 1, 350, 400, 2), generate_table(13, 1, 450, 500, 2), @@ -834,7 +834,7 @@ pub mod tests { Level { level_idx: 4, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(14, 1, 250, 300, 2), generate_table(15, 1, 350, 400, 2), generate_table(16, 1, 450, 500, 2), @@ -944,14 +944,14 @@ pub mod tests { Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![generate_table(0, 1, 400, 500, 2)], + sstable_infos: vec![generate_table(0, 1, 400, 500, 2)], total_file_size: 100, ..Default::default() }, Level { level_idx: 2, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(1, 1, 100, 200, 1), generate_table(2, 1, 600, 700, 1), ], @@ -961,7 +961,7 @@ pub mod tests { Level { level_idx: 3, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(3, 1, 100, 300, 2), generate_table(4, 1, 600, 800, 1), ], @@ -980,8 +980,8 @@ pub mod tests { let picker = MinOverlappingPicker::new(2, 3, 1000, 0, Arc::new(RangeOverlapStrategy::default())); let (select_files, target_files) = picker.pick_tables( - &levels[1].table_infos, - &levels[2].table_infos, + &levels[1].sstable_infos, + &levels[2].sstable_infos, &levels_handlers, ); let overlap_strategy = Arc::new(RangeOverlapStrategy::default()); @@ -989,7 +989,7 @@ pub mod tests { for sst in &select_files { overlap_info.update(sst); } - let range = overlap_info.check_multiple_overlap(&levels[0].table_infos); + let range = overlap_info.check_multiple_overlap(&levels[0].sstable_infos); assert!(range.is_empty()); assert_eq!(select_files.len(), 1); assert_eq!(target_files.len(), 1); @@ -1001,14 +1001,14 @@ pub mod tests { Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![generate_table(0, 1, 50, 100, 2)], // 50 + sstable_infos: vec![generate_table(0, 1, 50, 100, 2)], // 50 total_file_size: 50, ..Default::default() }, Level { level_idx: 2, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(1, 1, 101, 150, 1), // 50 ], total_file_size: 50, @@ -1017,7 +1017,7 @@ pub mod tests { Level { level_idx: 3, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(2, 1, 151, 200, 2), // 50 ], total_file_size: 50, @@ -1026,7 +1026,7 @@ pub mod tests { Level { level_idx: 4, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table(3, 1, 50, 300, 2), // 250 ], total_file_size: 250, diff --git a/src/meta/src/hummock/compaction/picker/mod.rs b/src/meta/src/hummock/compaction/picker/mod.rs index 3bed84fc57b9..cb366f09a3eb 100644 --- a/src/meta/src/hummock/compaction/picker/mod.rs +++ b/src/meta/src/hummock/compaction/picker/mod.rs @@ -71,7 +71,7 @@ impl CompactionInput { level_handlers[level.level_idx as usize].add_pending_task( task_id, self.target_level, - &level.table_infos, + &level.sstable_infos, ); } else { has_l0 = true; @@ -82,7 +82,7 @@ impl CompactionInput { .input_levels .iter() .filter(|level| level.level_idx == 0) - .flat_map(|level| level.table_infos.iter()); + .flat_map(|level| level.sstable_infos.iter()); level_handlers[0].add_pending_task(task_id, self.target_level, table_infos); } } diff --git a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs index b3e9eb75321a..9313d6ea4654 100644 --- a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs @@ -68,7 +68,7 @@ impl SpaceReclaimCompactionPicker { let l0 = &levels.l0; // only pick trivial reclaim sstables because this kind of task could be optimized and do not need send to compactor. for level in &l0.sub_levels { - for sst in &level.table_infos { + for sst in &level.sstable_infos { let exist_count = self.exist_table_count(sst); if exist_count == sst.table_ids.len() || level_handlers[0].is_pending_compact(&sst.sst_id) @@ -90,12 +90,12 @@ impl SpaceReclaimCompactionPicker { InputLevel { level_idx: level.level_idx, level_type: level.level_type, - table_infos: select_input_ssts, + sstable_infos: select_input_ssts, }, InputLevel { level_idx: 0, level_type: level.level_type, - table_infos: vec![], + sstable_infos: vec![], }, ], target_level: level.level_idx as usize, @@ -108,7 +108,7 @@ impl SpaceReclaimCompactionPicker { } while state.last_level <= levels.levels.len() { let mut is_trivial_task = true; - for sst in &levels.levels[state.last_level - 1].table_infos { + for sst in &levels.levels[state.last_level - 1].sstable_infos { let exist_count = self.exist_table_count(sst); let need_reclaim = exist_count < sst.table_ids.len(); let is_trivial_sst = exist_count == 0; @@ -146,12 +146,12 @@ impl SpaceReclaimCompactionPicker { InputLevel { level_idx: state.last_level as u32, level_type: levels.levels[state.last_level - 1].level_type, - table_infos: select_input_ssts, + sstable_infos: select_input_ssts, }, InputLevel { level_idx: state.last_level as u32, level_type: levels.levels[state.last_level - 1].level_type, - table_infos: vec![], + sstable_infos: vec![], }, ], target_level: state.last_level, @@ -215,7 +215,7 @@ mod test { Level { level_idx: 4, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table_with_ids_and_epochs(2, 1, 0, 100, 1, vec![2], 0, 0), generate_table_with_ids_and_epochs(3, 1, 101, 200, 1, vec![3], 0, 0), generate_table_with_ids_and_epochs(4, 1, 222, 300, 1, vec![4], 0, 0), @@ -232,7 +232,7 @@ mod test { ]; { - let sst_10 = levels[3].table_infos.get_mut(8).unwrap(); + let sst_10 = levels[3].sstable_infos.get_mut(8).unwrap(); assert_eq!(10, sst_10.sst_id); sst_10.key_range.right_exclusive = true; } @@ -271,8 +271,8 @@ mod test { assert_compaction_task(&task, &levels_handler); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 3); - assert_eq!(task.input.input_levels[0].table_infos.len(), 2); - levels_handler[4].add_pending_task(0, 4, &levels.levels[3].table_infos[5..6]); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 2); + levels_handler[4].add_pending_task(0, 4, &levels.levels[3].sstable_infos[5..6]); let task = selector .pick_compaction( 1, @@ -291,16 +291,16 @@ mod test { .unwrap(); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 4); - assert_eq!(task.input.input_levels[0].table_infos.len(), 5); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 5); let mut start_id = 2; - for sst in &task.input.input_levels[0].table_infos { + for sst in &task.input.input_levels[0].sstable_infos { assert_eq!(start_id, sst.sst_id); start_id += 1; } assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, @@ -309,7 +309,7 @@ mod test { // in this case, no files is pending, so it limit by max_space_reclaim_bytes let select_file_size: u64 = task.input.input_levels[0] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_size) .sum(); @@ -337,14 +337,14 @@ mod test { assert_compaction_task(&task, &levels_handler); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 4); - assert_eq!(task.input.input_levels[0].table_infos.len(), 4); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 4); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, compact_task::TaskType::SpaceReclaim )); let mut start_id = 8; - for sst in &task.input.input_levels[0].table_infos { + for sst in &task.input.input_levels[0].sstable_infos { assert_eq!(start_id, sst.sst_id); start_id += 1; } @@ -428,7 +428,7 @@ mod test { assert_compaction_task(&task, &levels_handler); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 3); - assert_eq!(task.input.input_levels[0].table_infos.len(), 2); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 2); assert_eq!(task.input.target_level, 3); assert!(matches!( task.compaction_task_type, @@ -473,9 +473,9 @@ mod test { assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 4); - assert_eq!(task.input.input_levels[0].table_infos.len(), *x); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), *x); let select_sst = &task.input.input_levels[0] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_id) .collect_vec(); @@ -483,7 +483,7 @@ mod test { assert_eq!(expect_task_sst_id_range[index], *select_sst); assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, @@ -534,9 +534,9 @@ mod test { assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 4); - assert_eq!(task.input.input_levels[0].table_infos.len(), *x); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), *x); let select_sst = &task.input.input_levels[0] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_id) .collect_vec(); @@ -544,7 +544,7 @@ mod test { assert_eq!(expect_task_sst_id_range[index], *select_sst); assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, diff --git a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs index 9099550232e3..9fb44a872b4b 100644 --- a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs @@ -59,7 +59,7 @@ impl TierCompactionPicker { continue; } - if level.table_infos.is_empty() { + if level.sstable_infos.is_empty() { continue; } @@ -70,7 +70,7 @@ impl TierCompactionPicker { let input_level = InputLevel { level_idx: 0, level_type: level.level_type, - table_infos: level.table_infos.clone(), + sstable_infos: level.sstable_infos.clone(), }; let mut select_level_inputs = vec![input_level]; @@ -84,7 +84,7 @@ impl TierCompactionPicker { ); let mut compaction_bytes = level.total_file_size; - let mut compact_file_count = level.table_infos.len() as u64; + let mut compact_file_count = level.sstable_infos.len() as u64; // Limit sstable file count to avoid using too much memory. let overlapping_max_compact_file_numer = self.config.level0_max_compact_file_number; @@ -102,11 +102,11 @@ impl TierCompactionPicker { } compaction_bytes += other.total_file_size; - compact_file_count += other.table_infos.len() as u64; + compact_file_count += other.sstable_infos.len() as u64; select_level_inputs.push(InputLevel { level_idx: 0, level_type: other.level_type, - table_infos: other.table_infos.clone(), + sstable_infos: other.sstable_infos.clone(), }); } @@ -216,7 +216,7 @@ pub mod tests { assert_eq!( ret.input_levels .iter() - .map(|i| i.table_infos.len()) + .map(|i| i.sstable_infos.len()) .sum::(), 7 ); diff --git a/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs index b4ab933931bd..049aceb471ba 100644 --- a/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs @@ -57,7 +57,7 @@ impl TombstoneReclaimCompactionPicker { while state.last_level <= levels.levels.len() { let mut select_input_ssts = vec![]; - for sst in &levels.levels[state.last_level - 1].table_infos { + for sst in &levels.levels[state.last_level - 1].sstable_infos { let need_reclaim = (sst.range_tombstone_count * 100 >= sst.total_key_count * self.range_delete_ratio) || (sst.stale_key_count * 100 >= sst.total_key_count * self.delete_ratio); @@ -78,12 +78,12 @@ impl TombstoneReclaimCompactionPicker { InputLevel { level_idx: state.last_level as u32, level_type: levels.levels[state.last_level - 1].level_type, - table_infos: vec![], + sstable_infos: vec![], } } else { let target_table_infos = self.overlap_strategy.check_base_level_overlap( &select_input_ssts, - &levels.levels[state.last_level].table_infos, + &levels.levels[state.last_level].sstable_infos, ); let mut pending_compact = false; for sst in &target_table_infos { @@ -99,24 +99,24 @@ impl TombstoneReclaimCompactionPicker { InputLevel { level_idx: (state.last_level + 1) as u32, level_type: levels.levels[state.last_level].level_type, - table_infos: target_table_infos, + sstable_infos: target_table_infos, } }; return Some(CompactionInput { select_input_size: select_input_ssts.iter().map(|sst| sst.sst_size).sum(), target_input_size: target_level - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_size) .sum(), - total_file_count: (select_input_ssts.len() + target_level.table_infos.len()) + total_file_count: (select_input_ssts.len() + target_level.sstable_infos.len()) as u64, target_level: target_level.level_idx as usize, input_levels: vec![ InputLevel { level_idx: state.last_level as u32, level_type: levels.levels[state.last_level - 1].level_type, - table_infos: select_input_ssts, + sstable_infos: select_input_ssts, }, target_level, ], @@ -168,25 +168,25 @@ pub mod tests { let mut sst = generate_table(3, 1, 201, 300, 1); sst.stale_key_count = 40; sst.total_key_count = 100; - levels.levels[1].table_infos.push(sst); + levels.levels[1].sstable_infos.push(sst); let ret = picker .pick_compaction(&levels, &levels_handler, &mut state) .unwrap(); assert_eq!(2, ret.input_levels.len()); - assert_eq!(3, ret.input_levels[0].table_infos[0].sst_id); + assert_eq!(3, ret.input_levels[0].sstable_infos[0].sst_id); let mut sst = generate_table(4, 1, 1, 100, 1); sst.stale_key_count = 30; sst.range_tombstone_count = 30; sst.total_key_count = 100; - levels.levels[0].table_infos.push(sst); + levels.levels[0].sstable_infos.push(sst); let picker = TombstoneReclaimCompactionPicker::new(strategy, 50, 10); let mut state = TombstoneReclaimPickerState::default(); let ret = picker .pick_compaction(&levels, &levels_handler, &mut state) .unwrap(); assert_eq!(2, ret.input_levels.len()); - assert_eq!(4, ret.input_levels[0].table_infos[0].sst_id); - assert_eq!(1, ret.input_levels[1].table_infos[0].sst_id); + assert_eq!(4, ret.input_levels[0].sstable_infos[0].sst_id); + assert_eq!(1, ret.input_levels[1].sstable_infos[0].sst_id); } } diff --git a/src/meta/src/hummock/compaction/picker/trivial_move_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/trivial_move_compaction_picker.rs index 458919cd8b71..6cd9316f89e9 100644 --- a/src/meta/src/hummock/compaction/picker/trivial_move_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/trivial_move_compaction_picker.rs @@ -96,12 +96,12 @@ impl TrivialMovePicker { InputLevel { level_idx: self.level as u32, level_type: LevelType::Nonoverlapping, - table_infos: vec![trivial_move_sst], + sstable_infos: vec![trivial_move_sst], }, InputLevel { level_idx: self.target_level as u32, level_type: LevelType::Nonoverlapping, - table_infos: vec![], + sstable_infos: vec![], }, ], target_level: self.target_level, diff --git a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs index 36ee06a3bb79..c8c790a64946 100644 --- a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs @@ -114,7 +114,7 @@ impl TtlReclaimCompactionPicker { let mut select_input_ssts = vec![]; let level_handler = &level_handlers[reclaimed_level.level_idx as usize]; - if reclaimed_level.table_infos.is_empty() { + if reclaimed_level.sstable_infos.is_empty() { // 1. not file to be picked state.clear(); return None; @@ -134,8 +134,8 @@ impl TtlReclaimCompactionPicker { if !state.valid() { // new round init key_range bound with table_infos - let first_sst = reclaimed_level.table_infos.first().unwrap(); - let last_sst = reclaimed_level.table_infos.last().unwrap(); + let first_sst = reclaimed_level.sstable_infos.first().unwrap(); + let last_sst = reclaimed_level.sstable_infos.last().unwrap(); let key_range_this_round = KeyRange { left: first_sst.key_range.left.clone(), @@ -148,7 +148,7 @@ impl TtlReclaimCompactionPicker { let current_epoch_physical_time = Epoch::now().physical_time(); - for sst in &reclaimed_level.table_infos { + for sst in &reclaimed_level.sstable_infos { let unmatched_sst = sst.key_range.sstable_overlap(&state.last_select_end_bound); if unmatched_sst @@ -182,12 +182,12 @@ impl TtlReclaimCompactionPicker { InputLevel { level_idx: reclaimed_level.level_idx, level_type: reclaimed_level.level_type, - table_infos: select_input_ssts, + sstable_infos: select_input_ssts, }, InputLevel { level_idx: reclaimed_level.level_idx, level_type: reclaimed_level.level_type, - table_infos: vec![], + sstable_infos: vec![], }, ], target_level: reclaimed_level.level_idx as usize, @@ -246,7 +246,7 @@ mod test { Level { level_idx: 4, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ generate_table_with_ids_and_epochs(2, 1, 0, 100, 1, vec![2], expired_epoch, 0), generate_table_with_ids_and_epochs( 3, @@ -347,7 +347,7 @@ mod test { ]; { - let sst_10 = levels[3].table_infos.get_mut(8).unwrap(); + let sst_10 = levels[3].sstable_infos.get_mut(8).unwrap(); assert_eq!(10, sst_10.sst_id); sst_10.key_range.right_exclusive = true; } @@ -392,16 +392,16 @@ mod test { assert_compaction_task(&task, &levels_handler); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 4); - assert_eq!(task.input.input_levels[0].table_infos.len(), 1); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 1); let mut start_id = 2; - for sst in &task.input.input_levels[0].table_infos { + for sst in &task.input.input_levels[0].sstable_infos { assert_eq!(start_id, sst.sst_id); start_id += 1; } assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, @@ -449,16 +449,16 @@ mod test { assert_eq!(task.input.input_levels[0].level_idx, 4); // test select index, picker will select file from state - assert_eq!(task.input.input_levels[0].table_infos.len(), 1); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 1); let mut start_id = 3; - for sst in &task.input.input_levels[0].table_infos { + for sst in &task.input.input_levels[0].sstable_infos { assert_eq!(start_id, sst.sst_id); start_id += 1; } assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, @@ -485,15 +485,15 @@ mod test { assert_compaction_task(&task, &levels_handler); assert_eq!(task.input.input_levels.len(), 2); assert_eq!(task.input.input_levels[0].level_idx, 4); - assert_eq!(task.input.input_levels[0].table_infos.len(), 1); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 1); assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, compact_task::TaskType::Ttl )); - for sst in &task.input.input_levels[0].table_infos { + for sst in &task.input.input_levels[0].sstable_infos { assert_eq!(start_id, sst.sst_id); start_id += 1; } @@ -548,12 +548,12 @@ mod test { assert_eq!(task.input.input_levels[0].level_idx, 4); // test table_option_filter - assert_eq!(task.input.input_levels[0].table_infos.len(), 1); - let select_sst = &task.input.input_levels[0].table_infos.first().unwrap(); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), 1); + let select_sst = &task.input.input_levels[0].sstable_infos.first().unwrap(); assert_eq!(select_sst.sst_id, 5); assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, @@ -662,9 +662,9 @@ mod test { assert_eq!(task.input.input_levels[0].level_idx, 4); // test table_option_filter - assert_eq!(task.input.input_levels[0].table_infos.len(), *x); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), *x); let select_sst = &task.input.input_levels[0] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_id) .collect_vec(); @@ -672,7 +672,7 @@ mod test { assert_eq!(expect_task_sst_id_range[index], *select_sst); assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, @@ -759,9 +759,9 @@ mod test { assert_eq!(task.input.input_levels[0].level_idx, 4); // test table_option_filter - assert_eq!(task.input.input_levels[0].table_infos.len(), *x); + assert_eq!(task.input.input_levels[0].sstable_infos.len(), *x); let select_sst = &task.input.input_levels[0] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_id) .collect_vec(); @@ -769,7 +769,7 @@ mod test { assert_eq!(expect_task_sst_id_range[index], *select_sst); assert_eq!(task.input.input_levels[1].level_idx, 4); - assert_eq!(task.input.input_levels[1].table_infos.len(), 0); + assert_eq!(task.input.input_levels[1].sstable_infos.len(), 0); assert_eq!(task.input.target_level, 4); assert!(matches!( task.compaction_task_type, diff --git a/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs b/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs index 5171c48ad9c3..7fbdf273ef6b 100644 --- a/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs +++ b/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs @@ -39,7 +39,7 @@ impl VnodeWatermarkCompactionPicker { ) -> Option { let level = levels.levels.last()?; let mut select_input_ssts = vec![]; - for sst_info in &level.table_infos { + for sst_info in &level.sstable_infos { if !level_handlers[level.level_idx as usize].is_pending_compact(&sst_info.sst_id) && should_delete_sst_by_watermark(sst_info, table_watermarks) { @@ -56,12 +56,12 @@ impl VnodeWatermarkCompactionPicker { InputLevel { level_idx: level.level_idx, level_type: level.level_type, - table_infos: select_input_ssts, + sstable_infos: select_input_ssts, }, InputLevel { level_idx: level.level_idx, level_type: level.level_type, - table_infos: vec![], + sstable_infos: vec![], }, ], target_level: level.level_idx as usize, diff --git a/src/meta/src/hummock/compaction/selector/level_selector.rs b/src/meta/src/hummock/compaction/selector/level_selector.rs index 321a3a65f713..3cd6f41e096a 100644 --- a/src/meta/src/hummock/compaction/selector/level_selector.rs +++ b/src/meta/src/hummock/compaction/selector/level_selector.rs @@ -209,7 +209,7 @@ impl DynamicLevelSelectorCore { .l0 .sub_levels .iter() - .map(|level| level.table_infos.len()) + .map(|level| level.sstable_infos.len()) .sum::() - handlers[0].get_pending_file_count(); @@ -224,7 +224,7 @@ impl DynamicLevelSelectorCore { .sub_levels .iter() .filter(|level| level.level_type == LevelType::Overlapping) - .map(|level| level.table_infos.len()) + .map(|level| level.sstable_infos.len()) .sum::(); if overlapping_file_count > 0 { // FIXME: use overlapping idle file count @@ -256,7 +256,7 @@ impl DynamicLevelSelectorCore { .sum::() .saturating_sub(handlers[0].get_pending_output_file_size(ctx.base_level as u32)); let base_level_size = levels.get_level(ctx.base_level).total_file_size; - let base_level_sst_count = levels.get_level(ctx.base_level).table_infos.len() as u64; + let base_level_sst_count = levels.get_level(ctx.base_level).sstable_infos.len() as u64; // size limit let non_overlapping_size_score = total_size * SCORE_BASE @@ -523,10 +523,10 @@ pub mod tests { assert_eq!(ctx.level_max_bytes[4], 1000); levels.levels[3] - .table_infos + .sstable_infos .append(&mut generate_tables(15..20, 2000..3000, 1, 400)); levels.levels[3].total_file_size = levels.levels[3] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_size) .sum::(); @@ -551,9 +551,9 @@ pub mod tests { levels.l0.sub_levels.clear(); levels.l0.total_file_size = 0; - levels.levels[0].table_infos = generate_tables(26..32, 0..1000, 1, 100); + levels.levels[0].sstable_infos = generate_tables(26..32, 0..1000, 1, 100); levels.levels[0].total_file_size = levels.levels[0] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_size) .sum::(); @@ -649,7 +649,7 @@ pub mod tests { levels_handlers[0].remove_task(1); levels_handlers[2].remove_task(1); levels.l0.sub_levels.clear(); - levels.levels[1].table_infos = generate_tables(20..30, 0..1000, 3, 10); + levels.levels[1].sstable_infos = generate_tables(20..30, 0..1000, 3, 10); let compaction = selector .pick_compaction( 2, @@ -671,7 +671,7 @@ pub mod tests { assert_eq!(compaction.input.target_level, 4); assert_eq!( compaction.input.input_levels[0] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_id) .collect_vec(), @@ -679,7 +679,7 @@ pub mod tests { ); assert_eq!( compaction.input.input_levels[1] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_id) .collect_vec(), diff --git a/src/meta/src/hummock/compaction/selector/mod.rs b/src/meta/src/hummock/compaction/selector/mod.rs index 02b50ff2ff67..2790f0a50bf9 100644 --- a/src/meta/src/hummock/compaction/selector/mod.rs +++ b/src/meta/src/hummock/compaction/selector/mod.rs @@ -143,7 +143,7 @@ pub mod tests { total_file_size: sst.sst_size, uncompressed_file_size: sst.uncompressed_file_size, sub_level_id: sst.sst_id, - table_infos: vec![sst], + sstable_infos: vec![sst], ..Default::default() }); } @@ -166,7 +166,7 @@ pub mod tests { level_type: LevelType::Nonoverlapping, total_file_size, sub_level_id, - table_infos, + sstable_infos: table_infos, uncompressed_file_size, ..Default::default() }); @@ -255,7 +255,7 @@ pub mod tests { Level { level_idx, level_type: LevelType::Nonoverlapping, - table_infos, + sstable_infos: table_infos, total_file_size, sub_level_id: 0, uncompressed_file_size, @@ -281,7 +281,7 @@ pub mod tests { total_file_size: table.sst_size, uncompressed_file_size: table.uncompressed_file_size, sub_level_id: idx as u64, - table_infos: vec![table], + sstable_infos: vec![table], ..Default::default() }) .collect_vec(), @@ -306,7 +306,7 @@ pub mod tests { .map(|sst| sst.uncompressed_file_size) .sum::(), sub_level_id: idx as u64, - table_infos: table, + sstable_infos: table, ..Default::default() }) .collect_vec(), @@ -337,7 +337,7 @@ pub mod tests { level_type: LevelType::Overlapping, total_file_size: table.iter().map(|table| table.sst_size).sum::(), sub_level_id: idx as u64, - table_infos: table.clone(), + sstable_infos: table.clone(), uncompressed_file_size: table .iter() .map(|sst| sst.uncompressed_file_size) @@ -359,7 +359,7 @@ pub mod tests { pub fn assert_compaction_task(compact_task: &CompactionTask, level_handlers: &[LevelHandler]) { for i in &compact_task.input.input_levels { - for t in &i.table_infos { + for t in &i.sstable_infos { assert!(level_handlers[i.level_idx as usize].is_pending_compact(&t.sst_id)); } } diff --git a/src/meta/src/hummock/level_handler.rs b/src/meta/src/hummock/level_handler.rs index 7ebc55eb1d3a..1009e69c66ca 100644 --- a/src/meta/src/hummock/level_handler.rs +++ b/src/meta/src/hummock/level_handler.rs @@ -65,18 +65,18 @@ impl LevelHandler { pub fn is_level_pending_compact(&self, level: &Level) -> bool { level - .table_infos + .sstable_infos .iter() .any(|table| self.compacting_files.contains_key(&table.sst_id)) } pub fn is_level_all_pending_compact(&self, level: &Level) -> bool { - if level.table_infos.is_empty() { + if level.sstable_infos.is_empty() { return false; } level - .table_infos + .sstable_infos .iter() .all(|table| self.compacting_files.contains_key(&table.sst_id)) } diff --git a/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs b/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs index ce1cdfb364e7..f925cdcd4ae5 100644 --- a/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs +++ b/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs @@ -140,12 +140,12 @@ impl HummockManager { for level_idx in 1..=max_level { let left_level = left_levels.get_level(level_idx); let right_level = right_levels.get_level(level_idx); - if left_level.table_infos.is_empty() || right_level.table_infos.is_empty() { + if left_level.sstable_infos.is_empty() || right_level.sstable_infos.is_empty() { continue; } - let left_last_sst = left_level.table_infos.last().unwrap().clone(); - let right_first_sst = right_level.table_infos.first().unwrap().clone(); + let left_last_sst = left_level.sstable_infos.last().unwrap().clone(); + let right_first_sst = right_level.sstable_infos.first().unwrap().clone(); let left_sst_id = left_last_sst.sst_id; let right_sst_id = right_first_sst.sst_id; let left_obj_id = left_last_sst.object_id; diff --git a/src/meta/src/hummock/manager/compaction/mod.rs b/src/meta/src/hummock/manager/compaction/mod.rs index be430dd17499..733983780d22 100644 --- a/src/meta/src/hummock/manager/compaction/mod.rs +++ b/src/meta/src/hummock/manager/compaction/mod.rs @@ -157,8 +157,11 @@ impl<'a> HummockVersionTransaction<'a> { for level in &compact_task.input_ssts { let level_idx = level.level_idx; - let mut removed_table_ids = - level.table_infos.iter().map(|sst| sst.sst_id).collect_vec(); + let mut removed_table_ids = level + .sstable_infos + .iter() + .map(|sst| sst.sst_id) + .collect_vec(); removed_table_ids_map .entry(level_idx) @@ -799,7 +802,7 @@ impl HummockManager { if !is_trivial_reclaim { compact_task .sorted_output_ssts - .clone_from(&compact_task.input_ssts[0].table_infos); + .clone_from(&compact_task.input_ssts[0].sstable_infos); } self.metrics .compact_frequency @@ -1077,12 +1080,12 @@ impl HummockManager { for input_level in &compact_task.input_ssts { let input_level: &InputLevel = input_level; let mut sst_ids: HashSet<_> = input_level - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_id) .collect(); fn filter_ssts(levels: &Level, sst_ids: &mut HashSet) { - for sst in &levels.table_infos { + for sst in &levels.sstable_infos { sst_ids.remove(&sst.sst_id); } } @@ -1196,7 +1199,7 @@ impl HummockManager { let input_sst_ids: HashSet = compact_task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter().map(|sst| sst.sst_id)) + .flat_map(|level| level.sstable_infos.iter().map(|sst| sst.sst_id)) .collect(); let input_level_ids: Vec = compact_task .input_ssts @@ -1461,7 +1464,7 @@ impl HummockManager { let mut table_size_info: HashMap = HashMap::default(); let mut existing_table_ids: HashSet = HashSet::default(); for input_ssts in &compact_task.input_ssts { - for sst in &input_ssts.table_infos { + for sst in &input_ssts.sstable_infos { existing_table_ids.extend(sst.table_ids.iter()); for table_id in &sst.table_ids { *table_size_info.entry(*table_id).or_default() += diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 9fa9b11a026c..c50b0cf35f73 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -232,7 +232,7 @@ async fn test_hummock_table() { .sub_levels .iter() .chain(levels.levels.iter()) - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .map(|info| info.object_id) .sorted() .cmp(original_tables.iter().map(|ot| ot.object_id).sorted()) @@ -1875,7 +1875,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { let compaction_task = get_manual_compact_task(hummock_manager.clone(), compaction_group_id, 0).await; let base_level: usize = 6; - assert_eq!(compaction_task.input_ssts[0].table_infos.len(), 1); + assert_eq!(compaction_task.input_ssts[0].sstable_infos.len(), 1); assert_eq!(compaction_task.target_level, base_level as u32); assert!(hummock_manager @@ -1924,7 +1924,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { current_version .get_compaction_group_levels(compaction_group_id) .levels[base_level - 1] - .table_infos + .sstable_infos .len(), 2 ); @@ -1942,7 +1942,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { current_version .get_compaction_group_levels(old_compaction_group_id) .levels[base_level - 1] - .table_infos + .sstable_infos .len(), 1 ); @@ -1951,7 +1951,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { current_version .get_compaction_group_levels(old_compaction_group_id) .levels[base_level - 1] - .table_infos[0] + .sstable_infos[0] .object_id, sst_1.sst_info.object_id + 1, ); @@ -1959,7 +1959,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { current_version .get_compaction_group_levels(old_compaction_group_id) .levels[base_level - 1] - .table_infos[0] + .sstable_infos[0] .table_ids, vec![101] ); @@ -1967,7 +1967,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { current_version .get_compaction_group_levels(new_compaction_group_id) .levels[base_level - 1] - .table_infos + .sstable_infos .len(), 2 ); @@ -1975,7 +1975,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { current_version .get_compaction_group_levels(new_compaction_group_id) .levels[base_level - 1] - .table_infos[0] + .sstable_infos[0] .table_ids, vec![100] ); @@ -1983,7 +1983,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { current_version .get_compaction_group_levels(new_compaction_group_id) .levels[base_level - 1] - .table_infos[1] + .sstable_infos[1] .table_ids, vec![100] ); @@ -2054,7 +2054,7 @@ async fn test_compaction_task_expiration_due_to_split_group() { let compaction_group_id = StaticCompactionGroupId::StateDefault.into(); let compaction_task = get_manual_compact_task(hummock_manager.clone(), compaction_group_id, 0).await; - assert_eq!(compaction_task.input_ssts[0].table_infos.len(), 2); + assert_eq!(compaction_task.input_ssts[0].sstable_infos.len(), 2); hummock_manager .split_compaction_group(compaction_group_id, &[100], 0) .await @@ -2073,7 +2073,7 @@ async fn test_compaction_task_expiration_due_to_split_group() { let compaction_task = get_manual_compact_task(hummock_manager.clone(), compaction_group_id, 0).await; - assert_eq!(compaction_task.input_ssts[0].table_infos.len(), 2); + assert_eq!(compaction_task.input_ssts[0].sstable_infos.len(), 2); hummock_manager .report_compact_task(compaction_task.task_id, TaskStatus::Success, vec![], None) .await @@ -2311,7 +2311,7 @@ async fn test_partition_level() { .iter() .map(|level| { level - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_size) .sum::() diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index 3a9ae5b098ae..cd039c956fec 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -342,7 +342,7 @@ fn rebuild_table_stats(version: &HummockVersion) -> HummockVersionStats { table_stats: Default::default(), }; for level in version.get_combined_levels() { - for sst in &level.table_infos { + for sst in &level.sstable_infos { let changes = estimate_table_stats(sst); add_prost_table_stats_map(&mut stats.table_stats, &changes); } @@ -537,7 +537,7 @@ mod tests { cg, Levels { levels: vec![Level { - table_infos: vec![sst.clone()], + sstable_infos: vec![sst.clone()], ..Default::default() }], ..Default::default() diff --git a/src/meta/src/hummock/metrics_utils.rs b/src/meta/src/hummock/metrics_utils.rs index ad39e2228fba..ee5bbeba0953 100644 --- a/src/meta/src/hummock/metrics_utils.rs +++ b/src/meta/src/hummock/metrics_utils.rs @@ -139,7 +139,7 @@ pub fn trigger_sst_stat( let mut sst_num = 0; current_version.level_iter(compaction_group_id, |level| { if level.level_idx == level_idx as u32 { - sst_num += level.table_infos.len(); + sst_num += level.sstable_infos.len(); } true }); diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 00cb52b34a0a..534f1125f064 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -102,7 +102,7 @@ pub async fn add_test_tables( compact_task .input_ssts .iter() - .map(|i| i.table_infos.len()) + .map(|i| i.sstable_infos.len()) .sum::(), 3 ); @@ -283,7 +283,7 @@ pub fn get_sorted_committed_object_ids( .levels .iter() .chain(levels.l0.sub_levels.iter()) - .flat_map(|levels| levels.table_infos.iter().map(|info| info.object_id)) + .flat_map(|levels| levels.sstable_infos.iter().map(|info| info.object_id)) .sorted() .collect_vec() } diff --git a/src/meta/src/manager/diagnose.rs b/src/meta/src/manager/diagnose.rs index afacce8dfdec..b6aa136395e6 100644 --- a/src/meta/src/manager/diagnose.rs +++ b/src/meta/src/manager/diagnose.rs @@ -473,10 +473,10 @@ impl DiagnoseCommand { .on_current_version(|version| { for compaction_group in version.levels.values() { let mut visit_level = |level: &Level| { - sst_num += level.table_infos.len(); + sst_num += level.sstable_infos.len(); sst_total_file_size += - level.table_infos.iter().map(|t| t.sst_size).sum::(); - for sst in &level.table_infos { + level.sstable_infos.iter().map(|t| t.sst_size).sum::(); + for sst in &level.sstable_infos { if sst.total_key_count == 0 { continue; } diff --git a/src/storage/hummock_sdk/src/compact.rs b/src/storage/hummock_sdk/src/compact.rs index b44a52e48491..22ca76d25684 100644 --- a/src/storage/hummock_sdk/src/compact.rs +++ b/src/storage/hummock_sdk/src/compact.rs @@ -69,7 +69,7 @@ pub fn compact_task_to_string(compact_task: &CompactTask) -> String { let mut dropped_table_ids = HashSet::new(); for level_entry in &compact_task.input_ssts { let tables: Vec = level_entry - .table_infos + .sstable_infos .iter() .map(|table| { for tid in &table.table_ids { @@ -159,9 +159,9 @@ pub fn statistics_compact_task(task: &CompactTask) -> CompactTaskStatistics { let mut total_uncompressed_file_size = 0; for level in &task.input_ssts { - total_file_count += level.table_infos.len() as u64; + total_file_count += level.sstable_infos.len() as u64; - level.table_infos.iter().for_each(|sst| { + level.sstable_infos.iter().for_each(|sst| { total_file_size += sst.file_size; total_uncompressed_file_size += sst.uncompressed_file_size; total_key_count += sst.total_key_count; @@ -206,7 +206,7 @@ pub fn estimate_memory_for_compact_task( for level in &task.input_ssts { if level.level_type == LevelType::Nonoverlapping { let mut cur_level_max_sst_meta_size = 0; - for sst in &level.table_infos { + for sst in &level.sstable_infos { let meta_size = sst.file_size - sst.meta_offset; task_max_sst_meta_ratio = std::cmp::max(task_max_sst_meta_ratio, meta_size * 100 / sst.file_size); @@ -214,7 +214,7 @@ pub fn estimate_memory_for_compact_task( } result += max_input_stream_estimated_memory + cur_level_max_sst_meta_size; } else { - for sst in &level.table_infos { + for sst in &level.sstable_infos { let meta_size = sst.file_size - sst.meta_offset; result += max_input_stream_estimated_memory + meta_size; task_max_sst_meta_ratio = diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index dc0b08d45299..38792d4ffc4e 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -74,10 +74,10 @@ pub fn summarize_group_deltas( delete_sst_levels.push(intra_level.level_idx); delete_sst_ids_set.extend(intra_level.removed_table_ids.iter().clone()); } - if !intra_level.inserted_table_infos.is_empty() { + if !intra_level.inserted_sstable_infos.is_empty() { insert_sst_level_id = intra_level.level_idx; insert_sub_level_id = intra_level.l0_sub_level_id; - insert_table_infos.extend(intra_level.inserted_table_infos.iter().cloned()); + insert_table_infos.extend(intra_level.inserted_sstable_infos.iter().cloned()); } new_vnode_partition_count = intra_level.vnode_partition_count; } @@ -162,7 +162,7 @@ impl HummockVersion { pub fn get_sst_infos(&self) -> impl Iterator { self.get_combined_levels() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .chain(self.table_change_log.values().flat_map(|change_log| { change_log.0.iter().flat_map(|epoch_change_log| { epoch_change_log @@ -188,7 +188,7 @@ impl HummockVersion { } }) .flat_map(|level| level.l0.sub_levels.iter().rev().chain(level.levels.iter())) - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .map(|s| s.sst_id) } @@ -209,7 +209,7 @@ impl HummockVersion { } }) .flat_map(|level| level.l0.sub_levels.iter().rev().chain(level.levels.iter())) - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .chain(self.table_change_log.values().flat_map(|change_log| { // TODO: optimization: strip table change log change_log.0.iter().flat_map(|epoch_change_log| { @@ -337,7 +337,7 @@ impl HummockVersion { .sub_levels .iter() .chain(parent_levels.levels.iter()) - .flat_map(|level| &level.table_infos) + .flat_map(|level| &level.sstable_infos) .map(|sst_info| { // `sst_info.table_ids` will never be empty. for table_id in &sst_info.table_ids { @@ -391,7 +391,7 @@ impl HummockVersion { let insert_table_infos = split_sst_info_for_level(&member_table_ids, sub_level, &mut new_sst_id); sub_level - .table_infos + .sstable_infos .extract_if(|sst_info| sst_info.table_ids.is_empty()) .for_each(|sst_info| { sub_level.total_file_size -= sst_info.sst_size; @@ -430,14 +430,14 @@ impl HummockVersion { .map(|sst| sst.uncompressed_file_size) .sum::(); cur_levels.levels[idx] - .table_infos + .sstable_infos .extend(insert_table_infos); cur_levels.levels[idx] - .table_infos + .sstable_infos .sort_by(|sst1, sst2| sst1.key_range.cmp(&sst2.key_range)); - assert!(can_concat(&cur_levels.levels[idx].table_infos)); + assert!(can_concat(&cur_levels.levels[idx].sstable_infos)); level - .table_infos + .sstable_infos .extract_if(|sst_info| sst_info.table_ids.is_empty()) .for_each(|sst_info| { level.total_file_size -= sst_info.sst_size; @@ -475,10 +475,10 @@ impl HummockVersion { // struct to reduce conventions. for group_delta in &group_deltas.group_deltas { if let GroupDelta::IntraLevel(intra_level) = group_delta { - if !intra_level.inserted_table_infos.is_empty() { + if !intra_level.inserted_sstable_infos.is_empty() { info.insert_sst_level = intra_level.level_idx; info.insert_sst_infos - .extend(intra_level.inserted_table_infos.iter().cloned()); + .extend(intra_level.inserted_sstable_infos.iter().cloned()); } if !intra_level.removed_table_ids.is_empty() { for id in &intra_level.removed_table_ids { @@ -497,7 +497,7 @@ impl HummockVersion { let group = self.levels.get(group_id).unwrap(); for l0_sub_level in &group.level0().sub_levels { - for sst_info in &l0_sub_level.table_infos { + for sst_info in &l0_sub_level.sstable_infos { if removed_l0_ssts.remove(&sst_info.sst_id) { info.delete_sst_object_ids.push(sst_info.object_id); } @@ -505,7 +505,7 @@ impl HummockVersion { } for level in &group.levels { if let Some(mut removed_level_ssts) = removed_ssts.remove(&level.level_idx) { - for sst_info in &level.table_infos { + for sst_info in &level.sstable_infos { if removed_level_ssts.remove(&sst_info.sst_id) { info.delete_sst_object_ids.push(sst_info.object_id); } @@ -623,7 +623,7 @@ impl HummockVersion { if let GroupDelta::IntraLevel(IntraLevelDelta { level_idx, l0_sub_level_id, - inserted_table_infos, + inserted_sstable_infos: inserted_table_infos, .. }) = group_delta { @@ -781,7 +781,7 @@ impl HummockVersion { levels.extend(group.l0.sub_levels.iter()); levels.extend(group.levels.iter()); for level in levels { - for table_info in &level.table_infos { + for table_info in &level.sstable_infos { if table_info.sst_id == table_info.object_id { continue; } @@ -895,7 +895,7 @@ impl Levels { "should find the level to insert into when applying compaction generated delta. sub level idx: {}, removed sst ids: {:?}, sub levels: {:?},", insert_sub_level_id, delete_sst_ids_set, l0.sub_levels.iter().map(|level| level.sub_level_id).collect_vec() ); - if l0.sub_levels[index].table_infos.is_empty() + if l0.sub_levels[index].sstable_infos.is_empty() && member_table_ids.len() == 1 && insert_table_infos.iter().all(|sst| { sst.table_ids.len() == 1 @@ -910,7 +910,7 @@ impl Levels { level_insert_ssts(&mut l0.sub_levels[index], insert_table_infos); } else { let idx = insert_sst_level_id as usize - 1; - if self.levels[idx].table_infos.is_empty() + if self.levels[idx].sstable_infos.is_empty() && insert_table_infos .iter() .all(|sst| sst.table_ids.len() == 1) @@ -928,7 +928,7 @@ impl Levels { if delete_sst_levels.iter().any(|level_id| *level_id == 0) { self.l0 .sub_levels - .retain(|level| !level.table_infos.is_empty()); + .retain(|level| !level.sstable_infos.is_empty()); self.l0.total_file_size = self .l0 .sub_levels @@ -952,13 +952,13 @@ impl Levels { for level_idx in delete_sst_levels { if *level_idx == 0 { for level in &self.l0.sub_levels { - level.table_infos.iter().for_each(|table| { + level.sstable_infos.iter().for_each(|table| { delete_sst_ids_set.remove(&table.sst_id); }); } } else { let idx = *level_idx as usize - 1; - self.levels[idx].table_infos.iter().for_each(|table| { + self.levels[idx].sstable_infos.iter().for_each(|table| { delete_sst_ids_set.remove(&table.sst_id); }); } @@ -976,7 +976,7 @@ pub fn build_initial_compaction_group_levels( levels.push(Level { level_idx: (l + 1) as u32, level_type: PbLevelType::Nonoverlapping, - table_infos: vec![], + sstable_infos: vec![], total_file_size: 0, sub_level_id: 0, uncompressed_file_size: 0, @@ -1005,7 +1005,7 @@ fn split_sst_info_for_level( // Remove SST from sub level may result in empty sub level. It will be purged // whenever another compaction task is finished. let mut insert_table_infos = vec![]; - for sst_info in &mut level.table_infos { + for sst_info in &mut level.sstable_infos { let removed_table_ids = sst_info .table_ids .iter() @@ -1058,7 +1058,7 @@ pub fn get_compaction_group_ssts( .chain(group_levels.levels.iter()) .flat_map(|level| { level - .table_infos + .sstable_infos .iter() .map(|table_info| (table_info.object_id, table_info.sst_id)) }) @@ -1084,7 +1084,7 @@ pub fn new_sub_level( Level { level_idx: 0, level_type, - table_infos, + sstable_infos: table_infos, total_file_size, sub_level_id, uncompressed_file_size, @@ -1104,17 +1104,17 @@ pub fn add_ssts_to_sub_level( l0.uncompressed_file_size += sst.uncompressed_file_size; }); l0.sub_levels[sub_level_idx] - .table_infos + .sstable_infos .extend(insert_table_infos); if l0.sub_levels[sub_level_idx].level_type == PbLevelType::Nonoverlapping { l0.sub_levels[sub_level_idx] - .table_infos + .sstable_infos .sort_by(|sst1, sst2| sst1.key_range.cmp(&sst2.key_range)); assert!( - can_concat(&l0.sub_levels[sub_level_idx].table_infos), + can_concat(&l0.sub_levels[sub_level_idx].sstable_infos), "sstable ids: {:?}", l0.sub_levels[sub_level_idx] - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_id) .collect_vec() @@ -1173,21 +1173,21 @@ fn level_delete_ssts( operand: &mut Level, delete_sst_ids_superset: &HashSet, ) -> bool { - let original_len = operand.table_infos.len(); + let original_len = operand.sstable_infos.len(); operand - .table_infos + .sstable_infos .retain(|table| !delete_sst_ids_superset.contains(&table.sst_id)); operand.total_file_size = operand - .table_infos + .sstable_infos .iter() .map(|table| table.sst_size) .sum::(); operand.uncompressed_file_size = operand - .table_infos + .sstable_infos .iter() .map(|table| table.uncompressed_file_size) .sum::(); - original_len != operand.table_infos.len() + original_len != operand.sstable_infos.len() } fn level_insert_ssts(operand: &mut Level, insert_table_infos: Vec) { @@ -1199,18 +1199,18 @@ fn level_insert_ssts(operand: &mut Level, insert_table_infos: Vec) .iter() .map(|sst| sst.uncompressed_file_size) .sum::(); - operand.table_infos.extend(insert_table_infos); + operand.sstable_infos.extend(insert_table_infos); operand - .table_infos + .sstable_infos .sort_by(|sst1, sst2| sst1.key_range.cmp(&sst2.key_range)); if operand.level_type == PbLevelType::Overlapping { operand.level_type = PbLevelType::Nonoverlapping; } assert!( - can_concat(&operand.table_infos), + can_concat(&operand.sstable_infos), "sstable ids: {:?}", operand - .table_infos + .sstable_infos .iter() .map(|sst| sst.sst_id) .collect_vec() @@ -1226,7 +1226,12 @@ pub fn object_size_map(version: &HummockVersion) -> HashMap Vec { if level.level_idx == 0 { level_identifier.push_str(format!("SUBLEVEL {}", level.sub_level_id).as_str()); // Ensure sub-level is not empty - if level.table_infos.is_empty() { + if level.sstable_infos.is_empty() { res.push(format!("{}: empty level", level_identifier)); } } else if level.level_type != PbLevelType::Nonoverlapping { @@ -1281,7 +1286,7 @@ pub fn validate_version(version: &HummockVersion) -> Vec { } let mut prev_table_info: Option<&SstableInfo> = None; - for table_info in &level.table_infos { + for table_info in &level.sstable_infos { // Ensure table_ids are sorted and unique if !table_info.table_ids.is_sorted_by(|a, b| a < b) { res.push(format!( @@ -1415,7 +1420,7 @@ mod tests { .l0 .sub_levels .push(Level { - table_infos: vec![SstableInfo { + sstable_infos: vec![SstableInfo { object_id: 11, sst_id: 11, ..Default::default() @@ -1426,7 +1431,7 @@ mod tests { // Add to non sub level version.levels.get_mut(&0).unwrap().levels.push(Level { - table_infos: vec![SstableInfo { + sstable_infos: vec![SstableInfo { object_id: 22, sst_id: 22, ..Default::default() @@ -1517,7 +1522,7 @@ mod tests { cg1.levels[0] = Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![SstableInfo { + sstable_infos: vec![SstableInfo { object_id: 1, sst_id: 1, ..Default::default() @@ -1584,7 +1589,7 @@ mod tests { left_levels.levels[0] = Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ gen_sst_info( 1, vec![3], @@ -1646,7 +1651,7 @@ mod tests { left_levels.l0.sub_levels.push(Level { level_idx: 0, - table_infos: vec![gen_sst_info( + sstable_infos: vec![gen_sst_info( 3, vec![3], FullKey::for_test( @@ -1672,7 +1677,7 @@ mod tests { left_levels.l0.sub_levels.push(Level { level_idx: 0, - table_infos: vec![gen_sst_info( + sstable_infos: vec![gen_sst_info( 3, vec![3], FullKey::for_test( @@ -1698,7 +1703,7 @@ mod tests { left_levels.l0.sub_levels.push(Level { level_idx: 0, - table_infos: vec![gen_sst_info( + sstable_infos: vec![gen_sst_info( 3, vec![3], FullKey::for_test( @@ -1725,7 +1730,7 @@ mod tests { right_levels.levels[0] = Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![ + sstable_infos: vec![ gen_sst_info( 1, vec![5], @@ -1787,7 +1792,7 @@ mod tests { right_levels.l0.sub_levels.push(Level { level_idx: 0, - table_infos: vec![gen_sst_info( + sstable_infos: vec![gen_sst_info( 3, vec![5], FullKey::for_test( @@ -1813,7 +1818,7 @@ mod tests { right_levels.l0.sub_levels.push(Level { level_idx: 0, - table_infos: vec![gen_sst_info( + sstable_infos: vec![gen_sst_info( 5, vec![5], FullKey::for_test( @@ -1839,7 +1844,7 @@ mod tests { right_levels.l0.sub_levels.push(Level { level_idx: 0, - table_infos: vec![gen_sst_info( + sstable_infos: vec![gen_sst_info( 3, vec![5], FullKey::for_test( diff --git a/src/storage/hummock_sdk/src/compaction_group/mod.rs b/src/storage/hummock_sdk/src/compaction_group/mod.rs index 94ef89b8046e..e97c9184c132 100644 --- a/src/storage/hummock_sdk/src/compaction_group/mod.rs +++ b/src/storage/hummock_sdk/src/compaction_group/mod.rs @@ -75,7 +75,7 @@ pub mod group_split { &mut left_levels.l0, right_sub_level.sub_level_id, right_sub_level.level_type, - right_sub_level.table_infos, + right_sub_level.sstable_infos, None, ); } @@ -98,11 +98,11 @@ pub mod group_split { .for_each(|sub_level| sub_level.vnode_partition_count = 0); for (idx, level) in right_levels.levels.into_iter().enumerate() { - if level.table_infos.is_empty() { + if level.sstable_infos.is_empty() { continue; } - let insert_table_infos = level.table_infos; + let insert_table_infos = level.sstable_infos; left_levels.levels[idx].total_file_size += insert_table_infos .iter() .map(|sst| sst.sst_size) @@ -113,20 +113,20 @@ pub mod group_split { .sum::(); left_levels.levels[idx] - .table_infos + .sstable_infos .extend(insert_table_infos); left_levels.levels[idx] - .table_infos + .sstable_infos .sort_by(|sst1, sst2| sst1.key_range.cmp(&sst2.key_range)); assert!( - can_concat(&left_levels.levels[idx].table_infos), + can_concat(&left_levels.levels[idx].sstable_infos), "{}", format!( - "left-group {} right-group {} left_levels.levels[{}].table_infos: {:?} level_idx {:?}", + "left-group {} right-group {} left_levels.levels[{}].sstable_infos: {:?} level_idx {:?}", left_levels.group_id, right_levels.group_id, idx, - left_levels.levels[idx].table_infos, + left_levels.levels[idx].sstable_infos, left_levels.levels[idx].level_idx ) ); diff --git a/src/storage/hummock_sdk/src/level.rs b/src/storage/hummock_sdk/src/level.rs index 762b5abd25ac..1506f51ba515 100644 --- a/src/storage/hummock_sdk/src/level.rs +++ b/src/storage/hummock_sdk/src/level.rs @@ -114,7 +114,7 @@ impl OverlappingLevel { pub struct LevelCommon { pub level_idx: u32, pub level_type: PbLevelType, - pub table_infos: Vec, + pub sstable_infos: Vec, pub total_file_size: u64, pub sub_level_id: u64, pub uncompressed_file_size: u64, @@ -131,7 +131,7 @@ where Self { level_idx: pb_level.level_idx, level_type: PbLevelType::try_from(pb_level.level_type).unwrap(), - table_infos: pb_level.table_infos.iter().map(Into::into).collect_vec(), + sstable_infos: pb_level.table_infos.iter().map(Into::into).collect_vec(), total_file_size: pb_level.total_file_size, sub_level_id: pb_level.sub_level_id, uncompressed_file_size: pb_level.uncompressed_file_size, @@ -148,7 +148,7 @@ where Self { level_idx: level.level_idx, level_type: level.level_type.into(), - table_infos: level.table_infos.iter().map(Into::into).collect_vec(), + table_infos: level.sstable_infos.iter().map(Into::into).collect_vec(), total_file_size: level.total_file_size, sub_level_id: level.sub_level_id, uncompressed_file_size: level.uncompressed_file_size, @@ -165,7 +165,11 @@ where Self { level_idx: level.level_idx, level_type: level.level_type.into(), - table_infos: level.table_infos.into_iter().map(Into::into).collect_vec(), + table_infos: level + .sstable_infos + .into_iter() + .map(Into::into) + .collect_vec(), total_file_size: level.total_file_size, sub_level_id: level.sub_level_id, uncompressed_file_size: level.uncompressed_file_size, @@ -182,7 +186,7 @@ where Self { level_idx: pb_level.level_idx, level_type: PbLevelType::try_from(pb_level.level_type).unwrap(), - table_infos: pb_level + sstable_infos: pb_level .table_infos .into_iter() .map(Into::into) @@ -200,7 +204,7 @@ impl Level { size_of::() + size_of::() + self - .table_infos + .sstable_infos .iter() .map(|sst| sst.estimated_encode_len()) .sum::() @@ -249,7 +253,7 @@ impl Levels { .sub_levels .iter() .chain(self.levels.iter()) - .map(|level| level.table_infos.len()) + .map(|level| level.sstable_infos.len()) .sum() } } @@ -360,7 +364,7 @@ where pub struct InputLevel { pub level_idx: u32, pub level_type: PbLevelType, - pub table_infos: Vec, + pub sstable_infos: Vec, } impl InputLevel { @@ -368,7 +372,7 @@ impl InputLevel { size_of::() + size_of::() + self - .table_infos + .sstable_infos .iter() .map(|sst| sst.estimated_encode_len()) .sum::() @@ -380,7 +384,7 @@ impl From for InputLevel { Self { level_idx: pb_input_level.level_idx, level_type: PbLevelType::try_from(pb_input_level.level_type).unwrap(), - table_infos: pb_input_level + sstable_infos: pb_input_level .table_infos .into_iter() .map(SstableInfo::from) @@ -394,7 +398,7 @@ impl From<&PbInputLevel> for InputLevel { Self { level_idx: pb_input_level.level_idx, level_type: PbLevelType::try_from(pb_input_level.level_type).unwrap(), - table_infos: pb_input_level + sstable_infos: pb_input_level .table_infos .iter() .map(SstableInfo::from) @@ -409,7 +413,7 @@ impl From for PbInputLevel { level_idx: input_level.level_idx, level_type: input_level.level_type.into(), table_infos: input_level - .table_infos + .sstable_infos .into_iter() .map(|sst| sst.into()) .collect_vec(), @@ -423,7 +427,7 @@ impl From<&InputLevel> for PbInputLevel { level_idx: input_level.level_idx, level_type: input_level.level_type.into(), table_infos: input_level - .table_infos + .sstable_infos .iter() .map(|sst| sst.into()) .collect_vec(), diff --git a/src/storage/hummock_sdk/src/time_travel.rs b/src/storage/hummock_sdk/src/time_travel.rs index 8859daa572d9..479df8b9d6f8 100644 --- a/src/storage/hummock_sdk/src/time_travel.rs +++ b/src/storage/hummock_sdk/src/time_travel.rs @@ -49,7 +49,7 @@ pub fn refill_version( } fn refill_level(level: &mut Level, sst_id_to_info: &HashMap) { - for s in &mut level.table_infos { + for s in &mut level.sstable_infos { refill_sstable_info(s, sst_id_to_info); } } diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 7be1cbdb834c..28a929e9cb21 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -517,7 +517,7 @@ impl HummockVersionDelta { group_deltas.group_deltas.iter().flat_map(|group_delta| { static EMPTY_VEC: Vec = Vec::new(); let sst_slice = if let GroupDelta::IntraLevel(level_delta) = &group_delta { - &level_delta.inserted_table_infos + &level_delta.inserted_sstable_infos } else { &EMPTY_VEC }; @@ -540,7 +540,7 @@ impl HummockVersionDelta { group_deltas.group_deltas.iter().flat_map(|group_delta| { static EMPTY_VEC: Vec = Vec::new(); let sst_slice = if let GroupDelta::IntraLevel(level_delta) = &group_delta { - &level_delta.inserted_table_infos + &level_delta.inserted_sstable_infos } else { &EMPTY_VEC }; @@ -576,7 +576,7 @@ impl HummockVersionDelta { group_deltas.group_deltas.iter().flat_map(|group_delta| { static EMPTY_VEC: Vec = Vec::new(); let sst_slice = if let GroupDelta::IntraLevel(level_delta) = &group_delta { - &level_delta.inserted_table_infos + &level_delta.inserted_sstable_infos } else { &EMPTY_VEC }; @@ -784,7 +784,7 @@ pub struct IntraLevelDeltaCommon { pub level_idx: u32, pub l0_sub_level_id: u64, pub removed_table_ids: Vec, - pub inserted_table_infos: Vec, + pub inserted_sstable_infos: Vec, pub vnode_partition_count: u32, } @@ -796,7 +796,7 @@ impl IntraLevelDelta { + size_of::() + self.removed_table_ids.len() * size_of::() + self - .inserted_table_infos + .inserted_sstable_infos .iter() .map(|sst| sst.estimated_encode_len()) .sum::() @@ -813,7 +813,7 @@ where level_idx: pb_intra_level_delta.level_idx, l0_sub_level_id: pb_intra_level_delta.l0_sub_level_id, removed_table_ids: pb_intra_level_delta.removed_table_ids, - inserted_table_infos: pb_intra_level_delta + inserted_sstable_infos: pb_intra_level_delta .inserted_table_infos .into_iter() .map(Into::into) @@ -833,7 +833,7 @@ where l0_sub_level_id: intra_level_delta.l0_sub_level_id, removed_table_ids: intra_level_delta.removed_table_ids, inserted_table_infos: intra_level_delta - .inserted_table_infos + .inserted_sstable_infos .into_iter() .map(Into::into) .collect_vec(), @@ -852,7 +852,7 @@ where l0_sub_level_id: intra_level_delta.l0_sub_level_id, removed_table_ids: intra_level_delta.removed_table_ids.clone(), inserted_table_infos: intra_level_delta - .inserted_table_infos + .inserted_sstable_infos .iter() .map(Into::into) .collect_vec(), @@ -870,7 +870,7 @@ where level_idx: pb_intra_level_delta.level_idx, l0_sub_level_id: pb_intra_level_delta.l0_sub_level_id, removed_table_ids: pb_intra_level_delta.removed_table_ids.clone(), - inserted_table_infos: pb_intra_level_delta + inserted_sstable_infos: pb_intra_level_delta .inserted_table_infos .iter() .map(Into::into) @@ -892,7 +892,7 @@ impl IntraLevelDelta { level_idx, l0_sub_level_id, removed_table_ids, - inserted_table_infos, + inserted_sstable_infos: inserted_table_infos, vnode_partition_count, } } diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 86f578d55bb3..483235aa2597 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -308,7 +308,7 @@ pub(crate) mod tests { .get_compaction_group_levels(compaction_group_id) .levels .iter() - .flat_map(|level| level.table_infos.clone()) + .flat_map(|level| level.sstable_infos.clone()) .collect_vec(); for output_table in &output_tables { let table = storage @@ -616,7 +616,7 @@ pub(crate) mod tests { .input_ssts .iter() .filter(|level| level.level_idx != compact_task.target_level) - .map(|level| level.table_infos.len()) + .map(|level| level.sstable_infos.len()) .sum::(), kv_count ); @@ -645,7 +645,7 @@ pub(crate) mod tests { let version: HummockVersion = hummock_manager_ref.get_current_version().await; let mut tables_from_version = vec![]; version.level_iter(compaction_group_id, |level| { - tables_from_version.extend(level.table_infos.iter().cloned()); + tables_from_version.extend(level.sstable_infos.iter().cloned()); true }); @@ -818,7 +818,7 @@ pub(crate) mod tests { compact_task .input_ssts .iter() - .map(|level| level.table_infos.len()) + .map(|level| level.sstable_infos.len()) .sum::(), kv_count, ); @@ -848,7 +848,7 @@ pub(crate) mod tests { let version: HummockVersion = hummock_manager_ref.get_current_version().await; let mut tables_from_version = vec![]; version.level_iter(compaction_group_id, |level| { - tables_from_version.extend(level.table_infos.iter().cloned()); + tables_from_version.extend(level.sstable_infos.iter().cloned()); true }); @@ -1007,7 +1007,7 @@ pub(crate) mod tests { compact_task .input_ssts .iter() - .map(|level| level.table_infos.len()) + .map(|level| level.sstable_infos.len()) .sum::(), kv_count, ); @@ -1046,7 +1046,7 @@ pub(crate) mod tests { .get_compaction_group_levels(compaction_group_id) .levels .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .collect::>(); let mut key_count = 0; @@ -1188,7 +1188,7 @@ pub(crate) mod tests { compact_task .input_ssts .iter() - .map(|level| level.table_infos.len()) + .map(|level| level.sstable_infos.len()) .sum::(), 129 ); @@ -1221,8 +1221,8 @@ pub(crate) mod tests { .levels .last() .unwrap(); - assert_eq!(1, output_level_info.table_infos.len()); - assert_eq!(252, output_level_info.table_infos[0].total_key_count); + assert_eq!(1, output_level_info.sstable_infos.len()); + assert_eq!(252, output_level_info.sstable_infos[0].total_key_count); } type KeyValue = (FullKey>, HummockValue>); @@ -1381,12 +1381,12 @@ pub(crate) mod tests { InputLevel { level_idx: 5, level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, - table_infos: ssts.drain(..select_file_count).collect_vec(), + sstable_infos: ssts.drain(..select_file_count).collect_vec(), }, InputLevel { level_idx: 6, level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, - table_infos: ssts, + sstable_infos: ssts, }, ], existing_table_ids: vec![1], @@ -1602,12 +1602,12 @@ pub(crate) mod tests { InputLevel { level_idx: 5, level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, - table_infos: sst_infos.drain(..1).collect_vec(), + sstable_infos: sst_infos.drain(..1).collect_vec(), }, InputLevel { level_idx: 6, level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, - table_infos: sst_infos, + sstable_infos: sst_infos, }, ], existing_table_ids: vec![1], @@ -1747,12 +1747,12 @@ pub(crate) mod tests { InputLevel { level_idx: 5, level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, - table_infos: sst_infos.drain(..1).collect_vec(), + sstable_infos: sst_infos.drain(..1).collect_vec(), }, InputLevel { level_idx: 6, level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, - table_infos: sst_infos, + sstable_infos: sst_infos, }, ], existing_table_ids: vec![1], diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index c59d33130d58..652ae9f5168e 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -2651,8 +2651,8 @@ async fn test_commit_multi_epoch() { assert_eq!(sub_levels.len(), 1); let sub_level = &sub_levels[0]; assert_eq!(sub_level.sub_level_id, epoch1); - assert_eq!(sub_level.table_infos.len(), 1); - assert_eq!(sub_level.table_infos[0].object_id, sst1_epoch1.object_id); + assert_eq!(sub_level.sstable_infos.len(), 1); + assert_eq!(sub_level.sstable_infos[0].object_id, sst1_epoch1.object_id); assert_eq!(version.visible_table_committed_epoch(), epoch1); @@ -2700,12 +2700,12 @@ async fn test_commit_multi_epoch() { assert_eq!(sub_levels.len(), 2); let sub_level = &sub_levels[0]; assert_eq!(sub_level.sub_level_id, epoch1); - assert_eq!(sub_level.table_infos.len(), 1); - assert_eq!(sub_level.table_infos[0].object_id, sst1_epoch1.object_id); + assert_eq!(sub_level.sstable_infos.len(), 1); + assert_eq!(sub_level.sstable_infos[0].object_id, sst1_epoch1.object_id); let sub_level = &sub_levels[1]; assert_eq!(sub_level.sub_level_id, epoch2); - assert_eq!(sub_level.table_infos.len(), 1); - assert_eq!(sub_level.table_infos[0].object_id, sst1_epoch2.object_id); + assert_eq!(sub_level.sstable_infos.len(), 1); + assert_eq!(sub_level.sstable_infos[0].object_id, sst1_epoch2.object_id); assert_eq!(version.visible_table_committed_epoch(), epoch2); @@ -2755,8 +2755,8 @@ async fn test_commit_multi_epoch() { assert_eq!(sub_levels.len(), 1); let sub_level1 = &sub_levels[0]; assert_eq!(sub_level1.sub_level_id, epoch1); - assert_eq!(sub_level1.table_infos.len(), 1); - assert_eq!(sub_level1.table_infos[0].object_id, sst2_epoch1.object_id); + assert_eq!(sub_level1.sstable_infos.len(), 1); + assert_eq!(sub_level1.sstable_infos[0].object_id, sst2_epoch1.object_id); assert_eq!(version.visible_table_committed_epoch(), epoch2); @@ -2793,12 +2793,12 @@ async fn test_commit_multi_epoch() { assert_eq!(sub_levels.len(), 2); let sub_level1 = &sub_levels[0]; assert_eq!(sub_level1.sub_level_id, epoch1); - assert_eq!(sub_level1.table_infos.len(), 1); - assert_eq!(sub_level1.table_infos[0].object_id, sst2_epoch1.object_id); + assert_eq!(sub_level1.sstable_infos.len(), 1); + assert_eq!(sub_level1.sstable_infos[0].object_id, sst2_epoch1.object_id); let sub_level2 = &sub_levels[1]; assert_eq!(sub_level2.sub_level_id, epoch2); - assert_eq!(sub_level2.table_infos.len(), 1); - assert_eq!(sub_level2.table_infos[0].object_id, sst2_epoch2.object_id); + assert_eq!(sub_level2.sstable_infos.len(), 1); + assert_eq!(sub_level2.sstable_infos[0].object_id, sst2_epoch2.object_id); assert_eq!(version.visible_table_committed_epoch(), epoch2); @@ -2837,32 +2837,32 @@ async fn test_commit_multi_epoch() { assert_eq!(sub_levels.len(), 3); let sub_level1 = &sub_levels[0]; assert_eq!(sub_level1.sub_level_id, epoch1); - assert_eq!(sub_level1.table_infos.len(), 1); - assert_eq!(sub_level1.table_infos[0].object_id, sst1_epoch1.object_id); + assert_eq!(sub_level1.sstable_infos.len(), 1); + assert_eq!(sub_level1.sstable_infos[0].object_id, sst1_epoch1.object_id); let sub_level2 = &sub_levels[1]; assert_eq!(sub_level2.sub_level_id, epoch2); - assert_eq!(sub_level2.table_infos.len(), 1); - assert_eq!(sub_level2.table_infos[0].object_id, sst1_epoch2.object_id); + assert_eq!(sub_level2.sstable_infos.len(), 1); + assert_eq!(sub_level2.sstable_infos[0].object_id, sst1_epoch2.object_id); let sub_level3 = &sub_levels[2]; assert_eq!(sub_level3.sub_level_id, epoch3); - assert_eq!(sub_level3.table_infos.len(), 1); - assert_eq!(sub_level3.table_infos[0].object_id, sst_epoch3.object_id); + assert_eq!(sub_level3.sstable_infos.len(), 1); + assert_eq!(sub_level3.sstable_infos[0].object_id, sst_epoch3.object_id); let new_cg = version.levels.get(&new_cg_id).unwrap(); let sub_levels = &new_cg.l0.sub_levels; assert_eq!(sub_levels.len(), 3); let sub_level1 = &sub_levels[0]; assert_eq!(sub_level1.sub_level_id, epoch1); - assert_eq!(sub_level1.table_infos.len(), 1); - assert_eq!(sub_level1.table_infos[0].object_id, sst2_epoch1.object_id); + assert_eq!(sub_level1.sstable_infos.len(), 1); + assert_eq!(sub_level1.sstable_infos[0].object_id, sst2_epoch1.object_id); let sub_level2 = &sub_levels[1]; assert_eq!(sub_level2.sub_level_id, epoch2); - assert_eq!(sub_level2.table_infos.len(), 1); - assert_eq!(sub_level2.table_infos[0].object_id, sst2_epoch2.object_id); + assert_eq!(sub_level2.sstable_infos.len(), 1); + assert_eq!(sub_level2.sstable_infos[0].object_id, sst2_epoch2.object_id); let sub_level3 = &sub_levels[1]; assert_eq!(sub_level3.sub_level_id, epoch2); - assert_eq!(sub_level3.table_infos.len(), 1); - assert_eq!(sub_level3.table_infos[0].object_id, sst2_epoch2.object_id); + assert_eq!(sub_level3.sstable_infos.len(), 1); + assert_eq!(sub_level3.sstable_infos[0].object_id, sst2_epoch2.object_id); assert_eq!(version.visible_table_committed_epoch(), epoch3); diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index 84cdf5513cde..81d18225262d 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -440,11 +440,11 @@ async fn test_syncpoints_get_in_delete_range_boundary() { let base_level = &version .get_compaction_group_levels(StaticCompactionGroupId::StateDefault.into()) .levels[4]; - assert_eq!(base_level.table_infos.len(), 3); - assert!(base_level.table_infos[0].key_range.right_exclusive); + assert_eq!(base_level.sstable_infos.len(), 3); + assert!(base_level.sstable_infos[0].key_range.right_exclusive); assert_eq!( - user_key(&base_level.table_infos[0].key_range.right), - user_key(&base_level.table_infos[1].key_range.left), + user_key(&base_level.sstable_infos[0].key_range.right), + user_key(&base_level.sstable_infos[1].key_range.left), ); storage.wait_version(version).await; let read_options = ReadOptions { diff --git a/src/storage/src/hummock/compactor/compaction_utils.rs b/src/storage/src/hummock/compactor/compaction_utils.rs index 3b032123f426..93048bf0385c 100644 --- a/src/storage/src/hummock/compactor/compaction_utils.rs +++ b/src/storage/src/hummock/compactor/compaction_utils.rs @@ -311,7 +311,7 @@ pub fn estimate_task_output_capacity(context: CompactorContext, task: &CompactTa let total_input_uncompressed_file_size = task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .map(|table| table.uncompressed_file_size) .sum::(); @@ -332,7 +332,7 @@ pub async fn check_compaction_result( let mut compact_table_ids = compact_task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .flat_map(|sst| sst.table_ids.clone()) .collect_vec(); compact_table_ids.sort(); @@ -349,24 +349,24 @@ pub async fn check_compaction_result( let mut table_iters = Vec::new(); for level in &compact_task.input_ssts { - if level.table_infos.is_empty() { + if level.sstable_infos.is_empty() { continue; } // Do not need to filter the table because manager has done it. if level.level_type == PbLevelType::Nonoverlapping { - debug_assert!(can_concat(&level.table_infos)); + debug_assert!(can_concat(&level.sstable_infos)); table_iters.push(ConcatSstableIterator::new( compact_task.existing_table_ids.clone(), - level.table_infos.clone(), + level.sstable_infos.clone(), KeyRange::inf(), context.sstable_store.clone(), Arc::new(TaskProgress::default()), context.storage_opts.compactor_iter_max_io_retry_times, )); } else { - for table_info in &level.table_infos { + for table_info in &level.sstable_infos { table_iters.push(ConcatSstableIterator::new( compact_task.existing_table_ids.clone(), vec![table_info.clone()], @@ -487,7 +487,7 @@ pub fn optimize_by_copy_block(compact_task: &CompactTask, context: &CompactorCon let sstable_infos = compact_task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .filter(|table_info| { let table_ids = &table_info.table_ids; table_ids @@ -517,7 +517,7 @@ pub fn optimize_by_copy_block(compact_task: &CompactTask, context: &CompactorCon let has_tombstone = compact_task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .any(|sst| sst.range_tombstone_count > 0); let has_ttl = compact_task .table_options @@ -527,14 +527,14 @@ pub fn optimize_by_copy_block(compact_task: &CompactTask, context: &CompactorCon let has_split_sst = compact_task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .any(|sst| sst.sst_id != sst.object_id); let compact_table_ids: HashSet = HashSet::from_iter( compact_task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .flat_map(|sst| sst.table_ids.clone()), ); let single_table = compact_table_ids.len() == 1; @@ -561,7 +561,7 @@ pub async fn generate_splits_for_task( let sstable_infos = compact_task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .filter(|table_info| { let table_ids = &table_info.table_ids; table_ids @@ -599,13 +599,13 @@ pub fn metrics_report_for_task(compact_task: &CompactTask, context: &CompactorCo .input_ssts .iter() .filter(|level| level.level_idx != compact_task.target_level) - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .collect_vec(); let target_table_infos = compact_task .input_ssts .iter() .filter(|level| level.level_idx == compact_task.target_level) - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .collect_vec(); let select_size = select_table_infos .iter() @@ -646,7 +646,7 @@ pub fn calculate_task_parallelism(compact_task: &CompactTask, context: &Compacto let sstable_infos = compact_task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .filter(|table_info| { let table_ids = &table_info.table_ids; table_ids diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index f6ad680beba0..74252646e0ad 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -89,7 +89,7 @@ impl CompactorRunner { let kv_count = task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .map(|sst| sst.total_key_count) .sum::() as usize; let use_block_based_filter = @@ -165,12 +165,12 @@ impl CompactorRunner { .compactor_iter_max_io_retry_times; let mut table_iters = Vec::new(); for level in &self.compact_task.input_ssts { - if level.table_infos.is_empty() { + if level.sstable_infos.is_empty() { continue; } let tables = level - .table_infos + .sstable_infos .iter() .filter(|table_info| { let table_ids = &table_info.table_ids; @@ -184,7 +184,7 @@ impl CompactorRunner { .collect_vec(); // Do not need to filter the table because manager has done it. if level.level_type == LevelType::Nonoverlapping { - debug_assert!(can_concat(&level.table_infos)); + debug_assert!(can_concat(&level.sstable_infos)); table_iters.push(ConcatSstableIterator::new( self.compact_task.existing_table_ids.clone(), tables, @@ -197,7 +197,7 @@ impl CompactorRunner { let sst_groups = partition_overlapping_sstable_infos(tables); tracing::warn!( "COMPACT A LARGE OVERLAPPING LEVEL: try to partition {} ssts with {} groups", - level.table_infos.len(), + level.sstable_infos.len(), sst_groups.len() ); for table_infos in sst_groups { @@ -322,7 +322,7 @@ pub async fn compact( compact_task .input_ssts .iter() - .flat_map(|level| level.table_infos.iter()) + .flat_map(|level| level.sstable_infos.iter()) .flat_map(|sst| sst.table_ids.clone()) .filter(|table_id| existing_table_ids.contains(table_id)), ); diff --git a/src/storage/src/hummock/compactor/fast_compactor_runner.rs b/src/storage/src/hummock/compactor/fast_compactor_runner.rs index 641d866e544b..7cb459e17e54 100644 --- a/src/storage/src/hummock/compactor/fast_compactor_runner.rs +++ b/src/storage/src/hummock/compactor/fast_compactor_runner.rs @@ -413,13 +413,13 @@ impl CompactorRunner { compact_task_to_string(&task) ); let left = Box::new(ConcatSstableIterator::new( - task.input_ssts[0].table_infos.clone(), + task.input_ssts[0].sstable_infos.clone(), context.sstable_store.clone(), task_progress.clone(), context.storage_opts.compactor_iter_max_io_retry_times, )); let right = Box::new(ConcatSstableIterator::new( - task.input_ssts[1].table_infos.clone(), + task.input_ssts[1].sstable_infos.clone(), context.sstable_store, task_progress.clone(), context.storage_opts.compactor_iter_max_io_retry_times, diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 7d0dc4984739..592fd9761ef1 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -612,14 +612,14 @@ impl HummockVersionReader { // the filter key needs to be encoded as well. assert!(committed_version.is_valid()); for level in committed_version.levels(read_options.table_id) { - if level.table_infos.is_empty() { + if level.sstable_infos.is_empty() { continue; } match level.level_type { LevelType::Overlapping | LevelType::Unspecified => { let sstable_infos = prune_overlapping_ssts( - &level.table_infos, + &level.sstable_infos, read_options.table_id, &single_table_key_range, ); @@ -645,12 +645,12 @@ impl HummockVersionReader { } LevelType::Nonoverlapping => { let mut table_info_idx = - search_sst_idx(&level.table_infos, full_key.user_key.as_ref()); + search_sst_idx(&level.sstable_infos, full_key.user_key.as_ref()); if table_info_idx == 0 { continue; } table_info_idx = table_info_idx.saturating_sub(1); - let ord = level.table_infos[table_info_idx] + let ord = level.sstable_infos[table_info_idx] .key_range .compare_right_with_user_key(full_key.user_key.as_ref()); // the case that the key falls into the gap between two ssts @@ -662,7 +662,7 @@ impl HummockVersionReader { local_stats.non_overlapping_get_count += 1; if let Some((data, data_epoch)) = get_from_sstable_info( self.sstable_store.clone(), - &level.table_infos[table_info_idx], + &level.sstable_infos[table_info_idx], full_key.to_ref(), &read_options, dist_key_hash, @@ -860,12 +860,13 @@ impl HummockVersionReader { let timer = Instant::now(); for level in committed.levels(read_options.table_id) { - if level.table_infos.is_empty() { + if level.sstable_infos.is_empty() { continue; } if level.level_type == LevelType::Nonoverlapping { - let table_infos = prune_nonoverlapping_ssts(&level.table_infos, user_key_range_ref); + let table_infos = + prune_nonoverlapping_ssts(&level.sstable_infos, user_key_range_ref); let sstables = table_infos .filter(|sstable_info| { sstable_info @@ -915,7 +916,7 @@ impl HummockVersionReader { } } else { let table_infos = prune_overlapping_ssts( - &level.table_infos, + &level.sstable_infos, read_options.table_id, &table_key_range, ); diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index b04c39769e90..c26a8605e02c 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -352,7 +352,7 @@ async fn start_replay( // The new meta should not have any data at this time for level in latest_version.levels.values() { level.levels.iter().for_each(|lvl| { - assert!(lvl.table_infos.is_empty()); + assert!(lvl.sstable_infos.is_empty()); assert_eq!(0, lvl.total_file_size); }); }