diff --git a/src/ctl/src/cmd_impl/hummock/list_version.rs b/src/ctl/src/cmd_impl/hummock/list_version.rs index 1c7ea9c40936..282f4edf4d60 100644 --- a/src/ctl/src/cmd_impl/hummock/list_version.rs +++ b/src/ctl/src/cmd_impl/hummock/list_version.rs @@ -34,7 +34,7 @@ pub async fn list_version( let l0 = levels.l0.as_mut().unwrap(); for sub_level in &mut l0.sub_levels { for t in &mut sub_level.table_infos { - t.key_range = None; + t.remove_key_range(); } } } @@ -42,7 +42,7 @@ pub async fn list_version( // l1 ~ lmax for level in &mut levels.levels { for t in &mut level.table_infos { - t.key_range = None; + t.remove_key_range(); } } }); @@ -63,18 +63,18 @@ pub async fn list_version( println!( "sub_level_id {} type {} sst_num {} size {}", sub_level.sub_level_id, - sub_level.level_type().as_str_name(), + sub_level.level_type.as_str_name(), sub_level.table_infos.len(), sub_level.total_file_size ) } } - for level in levels.get_levels() { + for level in &levels.levels { println!( "level_idx {} type {} sst_num {} size {}", level.level_idx, - level.level_type().as_str_name(), + level.level_type.as_str_name(), level.table_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 4d75912e0d4d..03a7b35a8519 100644 --- a/src/ctl/src/cmd_impl/hummock/sst_dump.rs +++ b/src/ctl/src/cmd_impl/hummock/sst_dump.rs @@ -28,9 +28,10 @@ use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAw use risingwave_common::util::value_encoding::{BasicSerde, EitherSerde, ValueRowDeserializer}; use risingwave_frontend::TableCatalog; use risingwave_hummock_sdk::key::FullKey; +use risingwave_hummock_sdk::level::Level; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_object_store::object::{ObjectMetadata, ObjectStoreImpl}; -use risingwave_pb::hummock::{Level, SstableInfo}; use risingwave_rpc_client::MetaClient; use risingwave_storage::hummock::value::HummockValue; use risingwave_storage::hummock::{ @@ -83,11 +84,11 @@ pub async fn sst_dump(context: &CtlContext, args: SstDumpArgs) -> anyhow::Result for level in version.get_combined_levels() { for sstable_info in &level.table_infos { if let Some(object_id) = &args.object_id { - if *object_id == sstable_info.get_object_id() { + if *object_id == sstable_info.object_id { print_level(level, sstable_info); sst_dump_via_sstable_store( &sstable_store, - sstable_info.get_object_id(), + sstable_info.object_id, sstable_info.meta_offset, sstable_info.file_size, &table_data, @@ -100,7 +101,7 @@ pub async fn sst_dump(context: &CtlContext, args: SstDumpArgs) -> anyhow::Result print_level(level, sstable_info); sst_dump_via_sstable_store( &sstable_store, - sstable_info.get_object_id(), + sstable_info.object_id, sstable_info.meta_offset, sstable_info.file_size, &table_data, @@ -161,7 +162,7 @@ pub async fn sst_dump(context: &CtlContext, args: SstDumpArgs) -> anyhow::Result } fn print_level(level: &Level, sst_info: &SstableInfo) { - println!("Level Type: {}", level.level_type); + println!("Level Type: {}", level.level_type.as_str_name()); println!("Level Idx: {}", level.level_idx); if level.level_idx == 0 { println!("L0 Sub-Level Idx: {}", level.sub_level_id); diff --git a/src/ctl/src/cmd_impl/hummock/validate_version.rs b/src/ctl/src/cmd_impl/hummock/validate_version.rs index e8f61a8c9835..7d0af4da9fee 100644 --- a/src/ctl/src/cmd_impl/hummock/validate_version.rs +++ b/src/ctl/src/cmd_impl/hummock/validate_version.rs @@ -20,11 +20,12 @@ use itertools::Itertools; use risingwave_common::util::epoch::Epoch; use risingwave_hummock_sdk::compaction_group::hummock_version_ext; use risingwave_hummock_sdk::key::{FullKey, UserKey}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{version_archive_dir, HummockSstableObjectId, HummockVersionId}; use risingwave_object_store::object::ObjectStoreRef; use risingwave_pb::hummock::group_delta::DeltaType; -use risingwave_pb::hummock::{HummockVersionArchive, SstableInfo}; +use risingwave_pb::hummock::HummockVersionArchive; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::hummock::value::HummockValue; use risingwave_storage::hummock::{Block, BlockHolder, BlockIterator, SstableStoreRef}; @@ -109,8 +110,7 @@ async fn print_user_key_in_version( .chain(cg.levels.iter()) { for sstable_info in &level.table_infos { - use risingwave_hummock_sdk::key_range::KeyRange; - let key_range: KeyRange = sstable_info.key_range.as_ref().unwrap().into(); + 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); if left_user_key.user_key > *target_key || *target_key > right_user_key.user_key { 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 f7d265485f70..bd54381c2bc9 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 @@ -92,7 +92,7 @@ fn remove_key_range_from_version(mut version: HummockVersion) -> HummockVersion .chain(cg.l0.as_mut().unwrap().sub_levels.iter_mut()) { for sst in &mut level.table_infos { - sst.key_range.take(); + sst.remove_key_range(); } } } @@ -107,7 +107,7 @@ fn version_to_compaction_group_rows(version: &HummockVersion) -> Vec Vec { for cg in version.levels.into_values() { for level in cg.levels.into_iter().chain(cg.l0.unwrap().sub_levels) { for sst in level.table_infos { - let key_range = sst.key_range.unwrap(); + let key_range = sst.key_range; sstables.push(RwHummockSstable { sstable_id: sst.sst_id as _, object_id: sst.object_id as _, @@ -125,8 +125,8 @@ fn version_to_sstable_rows(version: HummockVersion) -> Vec { level_id: level.level_idx as _, sub_level_id: (level.level_idx == 0).then_some(level.sub_level_id as _), level_type: level.level_type as _, - key_range_left: key_range.left, - key_range_right: key_range.right, + key_range_left: key_range.left.to_vec(), + key_range_right: key_range.right.to_vec(), right_exclusive: key_range.right_exclusive, file_size: sst.file_size as _, meta_offset: sst.meta_offset as _, diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs index 2608ca868119..20e1111e9c79 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs @@ -12,8 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; + use risingwave_common::types::{Fields, JsonbVal}; use risingwave_frontend_macro::system_catalog; +use risingwave_pb::hummock::hummock_version_delta::PbGroupDeltas; use serde_json::json; use crate::catalog::system_catalog::SysCatalogReaderImpl; @@ -41,7 +44,12 @@ async fn read(reader: &SysCatalogReaderImpl) -> Result>()) + .into(), }) .collect(); Ok(rows) diff --git a/src/meta/model_v2/src/hummock_sstable_info.rs b/src/meta/model_v2/src/hummock_sstable_info.rs index a9ca4f33361e..672d1f0e05a8 100644 --- a/src/meta/model_v2/src/hummock_sstable_info.rs +++ b/src/meta/model_v2/src/hummock_sstable_info.rs @@ -24,7 +24,7 @@ pub struct Model { #[sea_orm(primary_key, auto_increment = false)] pub sst_id: HummockSstableObjectId, pub object_id: HummockSstableObjectId, - pub sstable_info: SstableInfo, + pub sstable_info: SstableInfoV2Backend, } impl ActiveModelBehavior for ActiveModel {} @@ -32,4 +32,4 @@ impl ActiveModelBehavior for ActiveModel {} #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] pub enum Relation {} -crate::derive_from_blob!(SstableInfo, PbSstableInfo); +crate::derive_from_blob!(SstableInfoV2Backend, PbSstableInfo); diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index 10f53b9162e2..1e3330a2b53a 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -19,7 +19,7 @@ async-trait = "0.1" either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = { workspace = true } -prost ={ workspace = true } +prost = { workspace = true } rand = { workspace = true } regex = "1" risingwave_common = { workspace = true } diff --git a/src/meta/service/src/hummock_service.rs b/src/meta/service/src/hummock_service.rs index 3e5a26d2a777..579c6ffda7de 100644 --- a/src/meta/service/src/hummock_service.rs +++ b/src/meta/service/src/hummock_service.rs @@ -15,9 +15,11 @@ use std::collections::{HashMap, HashSet}; use std::time::Duration; +use compact_task::PbTaskStatus; use futures::StreamExt; use itertools::Itertools; use risingwave_common::catalog::{TableId, SYS_CATALOG_START_ID}; +use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::version::HummockVersionDelta; use risingwave_meta::manager::MetadataManager; use risingwave_pb::hummock::get_compaction_score_response::PickerInfo; @@ -85,7 +87,7 @@ impl HummockManagerService for HummockServiceImpl { let current_version = self.hummock_manager.get_current_version().await; Ok(Response::new(GetCurrentVersionResponse { status: None, - current_version: Some(current_version.to_protobuf()), + current_version: Some(current_version.into()), })) } @@ -101,7 +103,7 @@ impl HummockManagerService for HummockServiceImpl { )) .await?; Ok(Response::new(ReplayVersionDeltaResponse { - version: Some(version.to_protobuf()), + version: Some(version.into()), modified_compaction_groups: compaction_groups, })) } @@ -123,7 +125,7 @@ impl HummockManagerService for HummockServiceImpl { ) -> Result, Status> { let version = self.hummock_manager.disable_commit_epoch().await; Ok(Response::new(DisableCommitEpochResponse { - current_version: Some(version.to_protobuf()), + current_version: Some(version.into()), })) } @@ -139,8 +141,8 @@ impl HummockManagerService for HummockServiceImpl { let resp = ListVersionDeltasResponse { version_deltas: Some(PbHummockVersionDeltas { version_deltas: version_deltas - .iter() - .map(HummockVersionDelta::to_protobuf) + .into_iter() + .map(HummockVersionDelta::into) .collect(), }), }; @@ -234,8 +236,12 @@ impl HummockManagerService for HummockServiceImpl { // rewrite the key_range match request.key_range { - Some(key_range) => { - option.key_range = key_range; + Some(pb_key_range) => { + option.key_range = KeyRange { + left: pb_key_range.left.into(), + right: pb_key_range.right.into(), + right_exclusive: pb_key_range.right_exclusive, + }; } None => { @@ -426,7 +432,7 @@ impl HummockManagerService for HummockServiceImpl { let req = request.into_inner(); let version = self.hummock_manager.pin_version(req.context_id).await?; Ok(Response::new(PinVersionResponse { - pinned_version: Some(version.to_protobuf()), + pinned_version: Some(version.into()), })) } @@ -464,7 +470,7 @@ impl HummockManagerService for HummockServiceImpl { ) -> Result, Status> { let checkpoint_version = self.hummock_manager.get_checkpoint_version().await; Ok(Response::new(RiseCtlGetCheckpointVersionResponse { - checkpoint_version: Some(checkpoint_version.to_protobuf()), + checkpoint_version: Some(checkpoint_version.into()), })) } @@ -660,7 +666,10 @@ impl HummockManagerService for HummockServiceImpl { let request = request.into_inner(); let ret = self .hummock_manager - .cancel_compact_task(request.task_id, request.task_status()) + .cancel_compact_task( + request.task_id, + PbTaskStatus::try_from(request.task_status).unwrap(), + ) .await?; let response = Response::new(CancelCompactTaskResponse { ret }); diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index 5f0abbc8fe0e..96ab254211d1 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -352,7 +352,7 @@ impl NotificationServiceImpl { Ok(MetaSnapshot { tables, - hummock_version: Some(hummock_version.to_protobuf()), + hummock_version: Some(hummock_version.into()), version: Some(SnapshotVersion { catalog_version, ..Default::default() diff --git a/src/meta/src/backup_restore/restore.rs b/src/meta/src/backup_restore/restore.rs index 6165ba331d85..4a1c42805e40 100644 --- a/src/meta/src/backup_restore/restore.rs +++ b/src/meta/src/backup_restore/restore.rs @@ -97,7 +97,7 @@ async fn restore_hummock_version( ); let checkpoint_path = version_checkpoint_path(hummock_storage_directory); let checkpoint = PbHummockVersionCheckpoint { - version: Some(hummock_version.to_protobuf()), + version: Some(hummock_version.into()), // Ignore stale objects. Full GC will clear them. stale_objects: Default::default(), }; diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 50f8ab52b5fc..51fc54bf3980 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -1160,15 +1160,15 @@ fn collect_commit_epoch_info( for resp in resps { let ssts_iter = resp.synced_sstables.into_iter().map(|grouped| { let sst_info = grouped.sst.expect("field not None"); - sst_to_worker.insert(sst_info.get_object_id(), resp.worker_id); + sst_to_worker.insert(sst_info.object_id, resp.worker_id); LocalSstableInfo::new( - sst_info, + sst_info.into(), from_prost_table_stats_map(grouped.table_stats_map), ) }); synced_ssts.extend(ssts_iter); table_watermarks.push(resp.table_watermarks); - old_value_ssts.extend(resp.old_value_sstables); + old_value_ssts.extend(resp.old_value_sstables.into_iter().map(|s| s.into())); } let new_table_fragment_info = if let Command::CreateStreamingJob { info, .. } = &command_ctx.command { @@ -1215,10 +1215,7 @@ fn collect_commit_epoch_info( watermarks .into_iter() .map(|(table_id, watermarks)| { - ( - TableId::new(table_id), - TableWatermarks::from_protobuf(&watermarks), - ) + (TableId::new(table_id), TableWatermarks::from(&watermarks)) }) .collect() }) diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index f91f31b178b2..2ebbab619a79 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -17,6 +17,8 @@ pub mod compaction_config; mod overlap_strategy; use risingwave_common::catalog::{TableId, TableOption}; +use risingwave_hummock_sdk::compact_task::CompactTask; +use risingwave_hummock_sdk::level::Levels; use risingwave_pb::hummock::compact_task::{self, TaskType}; mod picker; @@ -30,8 +32,7 @@ use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_hummock_sdk::{CompactionGroupId, HummockCompactionTaskId}; use risingwave_pb::hummock::compaction_config::CompactionMode; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{CompactTask, CompactionConfig, LevelType}; +use risingwave_pb::hummock::{CompactionConfig, LevelType}; pub use selector::{CompactionSelector, CompactionSelectorContext}; use self::selector::{EmergencySelector, LocalSelectorStatistic}; @@ -145,12 +146,11 @@ impl CompactStatus { } pub fn is_trivial_move_task(task: &CompactTask) -> bool { - if task.task_type() != TaskType::Dynamic && task.task_type() != TaskType::Emergency { + if task.task_type != TaskType::Dynamic && task.task_type != TaskType::Emergency { return false; } - if task.input_ssts.len() != 2 - || task.input_ssts[0].level_type() != LevelType::Nonoverlapping + if task.input_ssts.len() != 2 || task.input_ssts[0].level_type != LevelType::Nonoverlapping { return false; } @@ -173,7 +173,7 @@ impl CompactStatus { pub fn is_trivial_reclaim(task: &CompactTask) -> bool { // Currently all VnodeWatermark tasks are trivial reclaim. - if task.task_type() == TaskType::VnodeWatermark { + if task.task_type == TaskType::VnodeWatermark { return true; } let exist_table_ids = HashSet::::from_iter(task.existing_table_ids.clone()); @@ -186,7 +186,6 @@ impl CompactStatus { }) } - /// Declares a task as either succeeded, failed or canceled. pub fn report_compact_task(&mut self, compact_task: &CompactTask) { for level in &compact_task.input_ssts { self.level_handlers[level.level_idx as usize].remove_task(compact_task.task_id); diff --git a/src/meta/src/hummock/compaction/overlap_strategy.rs b/src/meta/src/hummock/compaction/overlap_strategy.rs index 1a713cc40117..9a8617d0e927 100644 --- a/src/meta/src/hummock/compaction/overlap_strategy.rs +++ b/src/meta/src/hummock/compaction/overlap_strategy.rs @@ -17,9 +17,9 @@ use std::fmt::Debug; use std::ops::Range; use itertools::Itertools; -use risingwave_hummock_sdk::key_range::KeyRangeCommon; +use risingwave_hummock_sdk::key_range::{KeyRange, KeyRangeCommon}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::KeyComparator; -use risingwave_pb::hummock::{KeyRange, SstableInfo}; pub trait OverlapInfo: Debug { fn check_overlap(&self, a: &SstableInfo) -> bool; @@ -85,18 +85,14 @@ impl OverlapInfo for RangeOverlapInfo { match self.target_range.as_ref() { Some(key_range) => { let overlap_begin = others.partition_point(|table_status| { - table_status - .key_range - .as_ref() - .unwrap() - .compare_right_with(&key_range.left) + table_status.key_range.compare_right_with(&key_range.left) == cmp::Ordering::Less }); if overlap_begin >= others.len() { return overlap_begin..overlap_begin; } let overlap_end = others.partition_point(|table_status| { - key_range.compare_right_with(&table_status.key_range.as_ref().unwrap().left) + key_range.compare_right_with(&table_status.key_range.left) != cmp::Ordering::Less }); overlap_begin..overlap_end @@ -110,7 +106,7 @@ impl OverlapInfo for RangeOverlapInfo { Some(key_range) => { let overlap_begin = others.partition_point(|table_status| { KeyComparator::compare_encoded_full_key( - &table_status.key_range.as_ref().unwrap().left, + &table_status.key_range.left, &key_range.left, ) == cmp::Ordering::Less }); @@ -119,9 +115,7 @@ impl OverlapInfo for RangeOverlapInfo { } let mut overlap_end = overlap_begin; for table in &others[overlap_begin..] { - if key_range.compare_right_with(&table.key_range.as_ref().unwrap().right) - == cmp::Ordering::Less - { + if key_range.compare_right_with(&table.key_range.right) == cmp::Ordering::Less { break; } overlap_end += 1; @@ -133,7 +127,7 @@ impl OverlapInfo for RangeOverlapInfo { } fn update(&mut self, table: &SstableInfo) { - let other = table.key_range.as_ref().unwrap(); + let other = &table.key_range; if let Some(range) = self.target_range.as_mut() { range.full_key_extend(other); return; @@ -147,8 +141,7 @@ pub struct RangeOverlapStrategy {} impl OverlapStrategy for RangeOverlapStrategy { fn check_overlap(&self, a: &SstableInfo, b: &SstableInfo) -> bool { - let key_range = a.key_range.as_ref().unwrap(); - check_table_overlap(key_range, b) + check_table_overlap(&a.key_range, b) } fn create_overlap_info(&self) -> Box { @@ -157,6 +150,5 @@ impl OverlapStrategy for RangeOverlapStrategy { } fn check_table_overlap(key_range: &KeyRange, table: &SstableInfo) -> bool { - let other = table.key_range.as_ref().unwrap(); - key_range.sstable_overlap(other) + key_range.sstable_overlap(&table.key_range) } 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 62baa16a640e..8b2831d28e1d 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 @@ -17,9 +17,8 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_common::config::default::compaction_config; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockLevelsExt; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{CompactionConfig, InputLevel, Level, LevelType, OverlappingLevel}; +use risingwave_hummock_sdk::level::{InputLevel, Level, Levels, OverlappingLevel}; +use risingwave_pb::hummock::{CompactionConfig, LevelType}; use super::min_overlap_compaction_picker::NonOverlapSubLevelPicker; use super::{ @@ -52,7 +51,7 @@ impl CompactionPicker for LevelCompactionPicker { if l0.sub_levels.is_empty() { return None; } - if l0.sub_levels[0].level_type != LevelType::Nonoverlapping as i32 + if l0.sub_levels[0].level_type != LevelType::Nonoverlapping && l0.sub_levels[0].table_infos.len() > 1 { stats.skip_by_overlapping += 1; @@ -234,7 +233,7 @@ impl LevelCompactionPicker { .into_iter() .map(|table_infos| InputLevel { level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos, }) .collect_vec(); @@ -338,8 +337,8 @@ pub mod tests { .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].get_sst_id(), 4); - assert_eq!(ret.input_levels[1].table_infos[0].get_sst_id(), 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); ret.add_pending_task(0, &mut levels_handler); { @@ -350,13 +349,13 @@ pub mod tests { .unwrap(); assert_eq!(ret2.input_levels[0].table_infos.len(), 1); - assert_eq!(ret2.input_levels[0].table_infos[0].get_sst_id(), 6); - assert_eq!(ret2.input_levels[1].table_infos[0].get_sst_id(), 5); + assert_eq!(ret2.input_levels[0].table_infos[0].sst_id, 6); + assert_eq!(ret2.input_levels[1].table_infos[0].sst_id, 5); } levels.l0.as_mut().unwrap().sub_levels[0] .table_infos - .retain(|table| table.get_sst_id() != 4); + .retain(|table| table.sst_id != 4); levels.l0.as_mut().unwrap().total_file_size -= ret.input_levels[0].table_infos[0].file_size; levels_handler[0].remove_task(0); @@ -366,11 +365,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].get_sst_id(), 6); - assert_eq!(ret.input_levels[1].table_infos[0].get_sst_id(), 5); + 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].get_sst_id(), 3); - assert_eq!(ret.input_levels[2].table_infos[1].get_sst_id(), 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); ret.add_pending_task(1, &mut levels_handler); let mut local_stats = LocalPickerStatistic::default(); @@ -387,8 +386,8 @@ 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].get_sst_id(), 6); - assert_eq!(ret.input_levels[1].table_infos[0].get_sst_id(), 5); + 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); } #[test] @@ -406,7 +405,7 @@ pub mod tests { let levels = vec![Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(3, 1, 0, 50, 1), generate_table(4, 1, 150, 200, 1), @@ -447,7 +446,7 @@ pub mod tests { ret.input_levels[0] .table_infos .iter() - .map(|t| t.get_sst_id()) + .map(|t| t.sst_id) .collect_vec(), vec![1] ); @@ -456,7 +455,7 @@ pub mod tests { ret.input_levels[1] .table_infos .iter() - .map(|t| t.get_sst_id()) + .map(|t| t.sst_id) .collect_vec(), vec![3,] ); @@ -469,7 +468,7 @@ pub mod tests { let mut picker = create_compaction_picker_for_test(); let levels = vec![Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![], total_file_size: 0, sub_level_id: 0, @@ -533,7 +532,7 @@ pub mod tests { let mut levels = Levels { levels: vec![Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(1, 1, 100, 399, 2), generate_table(2, 1, 400, 699, 2), @@ -581,7 +580,7 @@ pub mod tests { ]); // We can set level_type only because the input above is valid. for s in &mut l0.sub_levels { - s.level_type = LevelType::Nonoverlapping as i32; + s.level_type = LevelType::Nonoverlapping; } let levels = Levels { l0: Some(l0), @@ -606,7 +605,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].get_sst_id(), 7); + assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 7); assert_eq!( 3, ret.input_levels.iter().filter(|l| l.level_idx == 0).count() @@ -634,7 +633,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].get_sst_id(), 6); + assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 6); assert_eq!( 2, ret.input_levels.iter().filter(|l| l.level_idx == 0).count() diff --git a/src/meta/src/hummock/compaction/picker/emergency_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/emergency_compaction_picker.rs index 4efcca28a981..f0fc3f237659 100644 --- a/src/meta/src/hummock/compaction/picker/emergency_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/emergency_compaction_picker.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use risingwave_pb::hummock::hummock_version::Levels; +use risingwave_hummock_sdk::level::Levels; use risingwave_pb::hummock::{CompactionConfig, LevelType}; use super::{ @@ -55,22 +55,20 @@ impl EmergencyCompactionPicker { let overlapping_count = l0 .sub_levels .iter() - .filter(|level| level.level_type == LevelType::Overlapping as i32) + .filter(|level| level.level_type == LevelType::Overlapping) .count(); let no_overlap_count = l0 .sub_levels .iter() .filter(|level| { - level.level_type == LevelType::Nonoverlapping as i32 - && level.vnode_partition_count == 0 + level.level_type == LevelType::Nonoverlapping && level.vnode_partition_count == 0 }) .count(); let partitioned_count = l0 .sub_levels .iter() .filter(|level| { - level.level_type == LevelType::Nonoverlapping as i32 - && level.vnode_partition_count > 0 + level.level_type == LevelType::Nonoverlapping && level.vnode_partition_count > 0 }) .count(); // We trigger `EmergencyCompactionPicker` only when some unexpected condition cause the number of l0 levels increase and the origin strategy 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 1261cca55089..dcd41073f05b 100644 --- a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs @@ -15,8 +15,8 @@ use std::sync::Arc; use risingwave_common::config::default::compaction_config; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{CompactionConfig, InputLevel, LevelType, OverlappingLevel}; +use risingwave_hummock_sdk::level::{InputLevel, Levels, OverlappingLevel}; +use risingwave_pb::hummock::{CompactionConfig, LevelType}; use super::min_overlap_compaction_picker::NonOverlapSubLevelPicker; use super::{ @@ -142,7 +142,7 @@ impl IntraCompactionPicker { } for (idx, level) in l0.sub_levels.iter().enumerate() { - if level.level_type() != LevelType::Nonoverlapping + if level.level_type != LevelType::Nonoverlapping || level.total_file_size > self.config.sub_level_max_compaction_bytes { continue; @@ -204,7 +204,7 @@ impl IntraCompactionPicker { } select_level_inputs.push(InputLevel { level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: level_select_sst, }); @@ -249,11 +249,11 @@ impl IntraCompactionPicker { let overlap_strategy = create_overlap_strategy(self.config.compaction_mode()); for (idx, level) in l0.sub_levels.iter().enumerate() { - if level.level_type == LevelType::Overlapping as i32 || idx + 1 >= l0.sub_levels.len() { + if level.level_type == LevelType::Overlapping || idx + 1 >= l0.sub_levels.len() { continue; } - if l0.sub_levels[idx + 1].level_type == LevelType::Overlapping as i32 { + if l0.sub_levels[idx + 1].level_type == LevelType::Overlapping { continue; } @@ -295,12 +295,12 @@ impl IntraCompactionPicker { let input_levels = vec![ InputLevel { level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![select_sst], }, InputLevel { level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![], }, ]; @@ -344,7 +344,7 @@ impl WholeLevelCompactionPicker { return None; } for (idx, level) in l0.sub_levels.iter().enumerate() { - if level.level_type() != LevelType::Nonoverlapping + if level.level_type != LevelType::Nonoverlapping || level.vnode_partition_count == partition_count { continue; @@ -417,7 +417,7 @@ impl WholeLevelCompactionPicker { #[cfg(test)] pub mod tests { - use risingwave_pb::hummock::Level; + use risingwave_hummock_sdk::level::Level; use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; @@ -444,7 +444,7 @@ pub mod tests { // compacting_key_range. let levels = vec![Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![], ..Default::default() }]; @@ -491,7 +491,7 @@ pub mod tests { let mut levels = Levels { levels: vec![Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![generate_table(3, 1, 200, 300, 2)], ..Default::default() }], @@ -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].get_table_infos()); + levels_handler[1].add_pending_task(100, 1, &levels.levels[0].table_infos); let config = Arc::new( CompactionConfigBuilder::new() .level0_sub_level_compact_level_count(1) @@ -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].get_table_infos()); + levels_handler[1].add_pending_task(100, 1, &levels.levels[0].table_infos); let config = Arc::new( CompactionConfigBuilder::new() .level0_sub_level_compact_level_count(1) @@ -610,9 +610,9 @@ pub mod tests { 3 ); - assert_eq!(4, ret.input_levels[0].table_infos[0].get_sst_id()); - assert_eq!(3, ret.input_levels[1].table_infos[0].get_sst_id()); - assert_eq!(1, ret.input_levels[2].table_infos[0].get_sst_id()); + 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); // will pick sst [2, 6, 5] let ret2 = picker @@ -627,9 +627,9 @@ pub mod tests { 3 ); - assert_eq!(5, ret2.input_levels[0].table_infos[0].get_sst_id()); - assert_eq!(6, ret2.input_levels[1].table_infos[0].get_sst_id()); - assert_eq!(2, ret2.input_levels[2].table_infos[0].get_sst_id()); + assert_eq!(5, ret2.input_levels[0].table_infos[0].sst_id); + assert_eq!(6, ret2.input_levels[1].table_infos[0].sst_id); + assert_eq!(2, ret2.input_levels[2].table_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].get_table_infos()); + levels_handler[1].add_pending_task(100, 1, &levels.levels[0].table_infos); let config = Arc::new( CompactionConfigBuilder::new() .level0_sub_level_compact_level_count(1) @@ -681,9 +681,9 @@ pub mod tests { 3 ); - assert_eq!(11, ret.input_levels[0].table_infos[0].get_sst_id()); - assert_eq!(9, ret.input_levels[1].table_infos[0].get_sst_id()); - assert_eq!(7, ret.input_levels[2].table_infos[0].get_sst_id()); + 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); let ret2 = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) @@ -697,9 +697,9 @@ pub mod tests { 3 ); - assert_eq!(5, ret2.input_levels[0].table_infos[0].get_sst_id()); - assert_eq!(10, ret2.input_levels[1].table_infos[0].get_sst_id()); - assert_eq!(2, ret2.input_levels[2].table_infos[0].get_sst_id()); + 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); } } @@ -732,7 +732,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].get_table_infos()); + levels_handler[1].add_pending_task(100, 1, &levels.levels[0].table_infos); let mut local_stats = LocalPickerStatistic::default(); let ret = picker.pick_compaction(&levels, &levels_handler, &mut local_stats); assert!(ret.is_none()); @@ -757,20 +757,20 @@ pub mod tests { .is_none()); // Cannot trivial move because latter sub-level is overlapping - levels.l0.as_mut().unwrap().sub_levels[0].level_type = LevelType::Nonoverlapping as i32; - levels.l0.as_mut().unwrap().sub_levels[1].level_type = LevelType::Overlapping as i32; + levels.l0.as_mut().unwrap().sub_levels[0].level_type = LevelType::Nonoverlapping; + levels.l0.as_mut().unwrap().sub_levels[1].level_type = LevelType::Overlapping; let ret = picker.pick_compaction(&levels, &levels_handler, &mut local_stats); assert!(ret.is_none()); // Cannot trivial move because former sub-level is overlapping - levels.l0.as_mut().unwrap().sub_levels[0].level_type = LevelType::Overlapping as i32; - levels.l0.as_mut().unwrap().sub_levels[1].level_type = LevelType::Nonoverlapping as i32; + levels.l0.as_mut().unwrap().sub_levels[0].level_type = LevelType::Overlapping; + levels.l0.as_mut().unwrap().sub_levels[1].level_type = LevelType::Nonoverlapping; let ret = picker.pick_compaction(&levels, &levels_handler, &mut local_stats); assert!(ret.is_none()); // trivial move - levels.l0.as_mut().unwrap().sub_levels[0].level_type = LevelType::Nonoverlapping as i32; - levels.l0.as_mut().unwrap().sub_levels[1].level_type = LevelType::Nonoverlapping as i32; + levels.l0.as_mut().unwrap().sub_levels[0].level_type = LevelType::Nonoverlapping; + levels.l0.as_mut().unwrap().sub_levels[1].level_type = LevelType::Nonoverlapping; let ret = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); @@ -860,15 +860,11 @@ pub mod tests { let input = ret.as_ref().unwrap(); assert_eq!(input.input_levels.len(), 2); assert_ne!( - levels.l0.as_ref().unwrap().get_sub_levels()[0] - .table_infos - .len(), + levels.l0.as_ref().unwrap().sub_levels[0].table_infos.len(), input.input_levels[0].table_infos.len() ); assert_ne!( - levels.l0.as_ref().unwrap().get_sub_levels()[1] - .table_infos - .len(), + levels.l0.as_ref().unwrap().sub_levels[1].table_infos.len(), input.input_levels[1].table_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 7919918ee73f..251a0a694b1a 100644 --- a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs @@ -16,9 +16,9 @@ use std::collections::HashSet; use std::sync::Arc; use itertools::Itertools; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockLevelsExt; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{InputLevel, Level, LevelType, OverlappingLevel, SstableInfo}; +use risingwave_hummock_sdk::level::{InputLevel, Level, Levels, OverlappingLevel}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_pb::hummock::LevelType; use super::{CompactionInput, CompactionPicker, LocalPickerStatistic}; use crate::hummock::compaction::overlap_strategy::{ @@ -163,7 +163,7 @@ impl ManualCompactionPicker { input_levels.reverse(); input_levels.push(InputLevel { level_idx: self.target_level as u32, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: target_input_ssts, }); @@ -181,7 +181,7 @@ impl ManualCompactionPicker { let mut hint_sst_ids: HashSet = HashSet::new(); hint_sst_ids.extend(self.option.sst_ids.iter()); let tmp_sst_info = SstableInfo { - key_range: Some(self.option.key_range.clone()), + key_range: self.option.key_range.clone(), ..Default::default() }; if self @@ -233,7 +233,7 @@ impl CompactionPicker for ManualCompactionPicker { hint_sst_ids.extend(self.option.sst_ids.iter()); let mut tmp_sst_info = SstableInfo::default(); let mut range_overlap_info = RangeOverlapInfo::default(); - tmp_sst_info.key_range = Some(self.option.key_range.clone()); + tmp_sst_info.key_range = self.option.key_range.clone(); range_overlap_info.update(&tmp_sst_info); let level = self.option.level; let target_level = self.target_level; @@ -271,15 +271,13 @@ impl CompactionPicker for ManualCompactionPicker { .get_level(level) .table_infos .iter() - .find_position(|p| { - p.get_sst_id() == select_input_ssts.first().unwrap().get_sst_id() - }) + .find_position(|p| p.sst_id == select_input_ssts.first().unwrap().sst_id) .unwrap(); let (right, _) = levels .get_level(level) .table_infos .iter() - .find_position(|p| p.get_sst_id() == select_input_ssts.last().unwrap().get_sst_id()) + .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(); vec![] @@ -328,9 +326,10 @@ impl CompactionPicker for ManualCompactionPicker { pub mod tests { use std::collections::{BTreeSet, HashMap}; + use bytes::Bytes; + use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_pb::hummock::compact_task; - pub use risingwave_pb::hummock::KeyRange; use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; @@ -379,7 +378,7 @@ pub mod tests { let levels = vec![ Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(0, 1, 0, 100, 1), generate_table(1, 1, 101, 200, 1), @@ -389,7 +388,7 @@ pub mod tests { }, Level { level_idx: 2, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(4, 1, 0, 100, 1), generate_table(5, 1, 101, 150, 1), @@ -417,8 +416,8 @@ pub mod tests { let option = ManualCompactionOption { level: 1, key_range: KeyRange { - left: iterator_test_key_of_epoch(1, 0, 1), - right: iterator_test_key_of_epoch(1, 201, 1), + left: Bytes::from(iterator_test_key_of_epoch(1, 0, 1)), + right: Bytes::from(iterator_test_key_of_epoch(1, 201, 1)), right_exclusive: false, }, ..Default::default() @@ -510,8 +509,8 @@ pub mod tests { sst_ids: vec![], level: 1, key_range: KeyRange { - left: iterator_test_key_of_epoch(1, 101, 1), - right: iterator_test_key_of_epoch(1, 199, 1), + left: Bytes::from(iterator_test_key_of_epoch(1, 101, 1)), + right: Bytes::from(iterator_test_key_of_epoch(1, 199, 1)), right_exclusive: false, }, internal_table_id: HashSet::from([2]), @@ -554,7 +553,7 @@ pub mod tests { let mut levels = vec![ Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(3, 1, 0, 100, 1), generate_table(4, 2, 2000, 3000, 1), @@ -563,7 +562,7 @@ pub mod tests { }, Level { level_idx: 2, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(1, 1, 0, 100, 1), generate_table(2, 2, 2000, 3000, 1), @@ -578,7 +577,7 @@ pub mod tests { for t in &mut l.table_infos { t.table_ids.clear(); if idx == 0 { - t.table_ids.push(((t.get_sst_id() % 2) + 1) as _); + t.table_ids.push(((t.sst_id % 2) + 1) as _); } else { t.table_ids.push(3); } @@ -603,7 +602,7 @@ pub mod tests { let l0 = generate_l0_overlapping_sublevels(vec![]); let levels = vec![Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(1, 1, 0, 100, 1), generate_table(2, 2, 100, 200, 1), @@ -627,7 +626,7 @@ pub mod tests { let l0 = generate_l0_nonoverlapping_sublevels(vec![]); let levels = vec![Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![], total_file_size: 0, sub_level_id: 0, @@ -644,8 +643,8 @@ pub mod tests { sst_ids: vec![1], level: 0, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), @@ -670,8 +669,8 @@ pub mod tests { sst_ids: vec![], level: 0, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), @@ -702,7 +701,7 @@ pub mod tests { result.input_levels[l] .table_infos .iter() - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), *e ); @@ -717,8 +716,8 @@ pub mod tests { sst_ids: vec![], level: 0, key_range: KeyRange { - left: iterator_test_key_of_epoch(1, 0, 2), - right: iterator_test_key_of_epoch(1, 200, 2), + left: Bytes::from(iterator_test_key_of_epoch(1, 0, 2)), + right: Bytes::from(iterator_test_key_of_epoch(1, 200, 2)), right_exclusive: false, }, internal_table_id: HashSet::default(), @@ -738,7 +737,7 @@ pub mod tests { result.input_levels[l] .table_infos .iter() - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), *e ); @@ -768,8 +767,8 @@ pub mod tests { sst_ids: sst_id_filter.clone(), level: *input_level as _, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), @@ -790,7 +789,7 @@ pub mod tests { result.input_levels[i] .table_infos .iter() - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), *e ); @@ -809,8 +808,8 @@ pub mod tests { sst_ids: vec![], level: input_level, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, // No matching internal table id. @@ -831,8 +830,8 @@ pub mod tests { sst_ids: vec![], level: input_level, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, // Include all sub level's table ids @@ -856,7 +855,7 @@ pub mod tests { .iter() .take(3) .flat_map(|s| s.table_infos.clone()) - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), vec![9, 10, 7, 8, 5, 6] ); @@ -864,7 +863,7 @@ pub mod tests { result.input_levels[3] .table_infos .iter() - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), vec![3] ); @@ -875,8 +874,8 @@ pub mod tests { sst_ids: vec![], level: input_level, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, // Only include bottom sub level's table id @@ -898,7 +897,7 @@ pub mod tests { .iter() .take(3) .flat_map(|s| s.table_infos.clone()) - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), vec![9, 10, 7, 8, 5, 6] ); @@ -906,7 +905,7 @@ pub mod tests { result.input_levels[3] .table_infos .iter() - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), vec![3] ); @@ -918,8 +917,8 @@ pub mod tests { sst_ids: vec![], level: input_level, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, // Only include partial top sub level's table id, but the whole top sub level is @@ -944,7 +943,7 @@ pub mod tests { .iter() .take(1) .flat_map(|s| s.table_infos.clone()) - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), vec![5, 6] ); @@ -952,7 +951,7 @@ pub mod tests { result.input_levels[1] .table_infos .iter() - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), vec![3] ); @@ -962,8 +961,8 @@ pub mod tests { sst_ids: vec![], level: input_level, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, // Only include bottom sub level's table id @@ -995,8 +994,8 @@ pub mod tests { sst_ids: vec![], level: input_level, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, // No matching internal table id. @@ -1018,8 +1017,8 @@ pub mod tests { sst_ids: vec![], level: input_level, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, // Only include partial input level's table id @@ -1047,7 +1046,7 @@ pub mod tests { result.input_levels[l] .table_infos .iter() - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), *e ); @@ -1070,8 +1069,8 @@ pub mod tests { sst_ids: sst_id_filter.clone(), level: *input_level as _, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), @@ -1090,7 +1089,7 @@ pub mod tests { result.input_levels[i] .table_infos .iter() - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), *e ); @@ -1116,8 +1115,8 @@ pub mod tests { sst_ids: sst_id_filter.clone(), level: *input_level as _, key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), @@ -1140,7 +1139,7 @@ pub mod tests { result.input_levels[i] .table_infos .iter() - .map(|s| s.get_sst_id()) + .map(|s| s.sst_id) .collect_vec(), *e ); @@ -1163,7 +1162,7 @@ pub mod tests { generate_level(3, vec![]), Level { level_idx: 4, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(2, 1, 0, 100, 1), generate_table(3, 1, 101, 200, 1), @@ -1186,8 +1185,8 @@ pub mod tests { let option = ManualCompactionOption { sst_ids: vec![0, 1], key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), @@ -1228,8 +1227,8 @@ pub mod tests { let option = ManualCompactionOption { sst_ids: vec![], key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), @@ -1280,7 +1279,7 @@ pub mod tests { ), Level { level_idx: 4, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(2, 1, 0, 100, 1), generate_table(3, 1, 101, 200, 1), @@ -1306,8 +1305,8 @@ pub mod tests { let option = ManualCompactionOption { sst_ids: vec![0, 1], key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), @@ -1350,8 +1349,8 @@ pub mod tests { let option = ManualCompactionOption { sst_ids: vec![], key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), 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 57dd5469d42a..c444f1f647eb 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 @@ -15,10 +15,9 @@ use std::sync::Arc; use risingwave_hummock_sdk::append_sstable_info_to_string; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockLevelsExt; -use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{InputLevel, Level, LevelType, SstableInfo}; +use risingwave_hummock_sdk::level::{InputLevel, Level, Levels}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_pb::hummock::LevelType; use super::{CompactionInput, CompactionPicker, LocalPickerStatistic}; use crate::hummock::compaction::overlap_strategy::OverlapStrategy; @@ -156,12 +155,12 @@ impl CompactionPicker for MinOverlappingPicker { input_levels: vec![ InputLevel { level_idx: self.level as u32, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: select_input_ssts, }, InputLevel { level_idx: self.target_level as u32, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: target_input_ssts, }, ], @@ -253,7 +252,7 @@ impl NonOverlapSubLevelPicker { // Pay attention to the order here: Make sure to select the lowest sub_level to meet the requirements of base compaction. If you break the assumption of this order, you need to redesign it. // TODO: Use binary selection to replace the step algorithm to optimize algorithm complexity 'expand_new_level: for (target_index, target_level) in levels.iter().enumerate().skip(1) { - if target_level.level_type() != LevelType::Nonoverlapping { + if target_level.level_type != LevelType::Nonoverlapping { break; } @@ -310,7 +309,7 @@ impl NonOverlapSubLevelPicker { } basic_overlap_info.update(other); - add_files_size += other.get_file_size(); + add_files_size += other.file_size; add_files_count += 1; } @@ -345,11 +344,7 @@ impl NonOverlapSubLevelPicker { // sort sst per level due to reverse expand ret.sstable_infos.iter_mut().for_each(|level_ssts| { - level_ssts.sort_by(|sst1, sst2| { - let a = sst1.key_range.as_ref().unwrap(); - let b = sst2.key_range.as_ref().unwrap(); - a.compare(b) - }); + level_ssts.sort_by(|sst1, sst2| sst1.key_range.cmp(&sst2.key_range)); }); } else { ret.total_file_count = 1; @@ -549,7 +544,7 @@ pub mod tests { let levels = vec![ Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(0, 1, 0, 100, 1), generate_table(1, 1, 101, 200, 1), @@ -559,7 +554,7 @@ pub mod tests { }, Level { level_idx: 2, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(4, 1, 0, 100, 1), generate_table(5, 1, 101, 150, 1), @@ -590,7 +585,7 @@ pub mod tests { 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].get_sst_id(), 2); + assert_eq!(ret.input_levels[0].table_infos[0].sst_id, 2); assert_eq!(ret.input_levels[1].table_infos.len(), 0); ret.add_pending_task(0, &mut level_handlers); @@ -600,18 +595,18 @@ pub mod tests { 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].get_sst_id(), 0); + 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].get_sst_id(), 4); + assert_eq!(ret.input_levels[1].table_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].get_sst_id(), 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].get_sst_id(), 5); + assert_eq!(ret.input_levels[1].table_infos[0].sst_id, 5); } #[test] @@ -621,7 +616,7 @@ pub mod tests { let levels = vec![ Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(0, 1, 50, 99, 2), generate_table(1, 1, 100, 149, 2), @@ -631,7 +626,7 @@ pub mod tests { }, Level { level_idx: 2, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(4, 1, 50, 199, 1), generate_table(5, 1, 200, 399, 1), @@ -663,11 +658,11 @@ pub mod tests { 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].get_sst_id(), 0); - assert_eq!(ret.input_levels[0].table_infos[1].get_sst_id(), 1); + 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[1].table_infos.len(), 1); - assert_eq!(ret.input_levels[1].table_infos[0].get_sst_id(), 4); + assert_eq!(ret.input_levels[1].table_infos[0].sst_id, 4); } #[test] @@ -675,7 +670,7 @@ pub mod tests { let levels = vec![ Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(0, 1, 50, 99, 2), generate_table(1, 1, 100, 149, 2), @@ -689,7 +684,7 @@ pub mod tests { }, Level { level_idx: 2, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(4, 1, 50, 199, 1), generate_table(5, 1, 200, 249, 1), @@ -702,7 +697,7 @@ pub mod tests { }, Level { level_idx: 3, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(11, 1, 250, 300, 2), generate_table(12, 1, 350, 400, 2), @@ -713,7 +708,7 @@ pub mod tests { }, Level { level_idx: 4, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(14, 1, 250, 300, 2), generate_table(15, 1, 350, 400, 2), @@ -781,7 +776,7 @@ pub mod tests { let levels = vec![ Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(0, 1, 50, 99, 2), generate_table(1, 1, 100, 149, 2), @@ -795,7 +790,7 @@ pub mod tests { }, Level { level_idx: 2, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(4, 1, 50, 99, 1), generate_table(5, 1, 150, 200, 1), @@ -808,7 +803,7 @@ pub mod tests { }, Level { level_idx: 3, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(11, 1, 250, 300, 2), generate_table(12, 1, 350, 400, 2), @@ -819,7 +814,7 @@ pub mod tests { }, Level { level_idx: 4, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(14, 1, 250, 300, 2), generate_table(15, 1, 350, 400, 2), @@ -885,7 +880,7 @@ pub mod tests { for plan in ret { let mut sst_id_set = BTreeSet::default(); for sst in &plan.sstable_infos { - sst_id_set.insert(sst[0].get_sst_id()); + sst_id_set.insert(sst[0].sst_id); } assert!(sst_id_set.len() <= max_file_count as usize); } @@ -909,7 +904,7 @@ pub mod tests { for plan in ret { let mut sst_id_set = BTreeSet::default(); for sst in &plan.sstable_infos { - sst_id_set.insert(sst[0].get_sst_id()); + sst_id_set.insert(sst[0].sst_id); } assert!(plan.sstable_infos.len() >= min_depth); } @@ -921,14 +916,14 @@ pub mod tests { let levels = [ Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![generate_table(0, 1, 400, 500, 2)], total_file_size: 100, ..Default::default() }, Level { level_idx: 2, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(1, 1, 100, 200, 1), generate_table(2, 1, 600, 700, 1), @@ -938,7 +933,7 @@ pub mod tests { }, Level { level_idx: 3, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(3, 1, 100, 300, 2), generate_table(4, 1, 600, 800, 1), @@ -978,14 +973,14 @@ pub mod tests { let levels = vec![ Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![generate_table(0, 1, 50, 100, 2)], // 50 total_file_size: 50, ..Default::default() }, Level { level_idx: 2, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(1, 1, 101, 150, 1), // 50 ], @@ -994,7 +989,7 @@ pub mod tests { }, Level { level_idx: 3, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(2, 1, 151, 200, 2), // 50 ], @@ -1003,7 +998,7 @@ pub mod tests { }, Level { level_idx: 4, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table(3, 1, 50, 300, 2), // 250 ], diff --git a/src/meta/src/hummock/compaction/picker/mod.rs b/src/meta/src/hummock/compaction/picker/mod.rs index 05289cf10403..3bed84fc57b9 100644 --- a/src/meta/src/hummock/compaction/picker/mod.rs +++ b/src/meta/src/hummock/compaction/picker/mod.rs @@ -32,8 +32,7 @@ pub use emergency_compaction_picker::EmergencyCompactionPicker; pub use intra_compaction_picker::IntraCompactionPicker; pub use manual_compaction_picker::ManualCompactionPicker; pub use min_overlap_compaction_picker::MinOverlappingPicker; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::InputLevel; +use risingwave_hummock_sdk::level::{InputLevel, Levels}; pub use space_reclaim_compaction_picker::{SpaceReclaimCompactionPicker, SpaceReclaimPickerState}; pub use tier_compaction_picker::TierCompactionPicker; pub use tombstone_reclaim_compaction_picker::{ 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 6d3b1717bf8b..8d69e85bcc48 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 @@ -14,8 +14,8 @@ use std::collections::HashSet; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{InputLevel, SstableInfo}; +use risingwave_hummock_sdk::level::{InputLevel, Levels}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use super::CompactionInput; use crate::hummock::level_handler::LevelHandler; @@ -173,9 +173,10 @@ mod test { use itertools::Itertools; use risingwave_common::catalog::TableId; + use risingwave_hummock_sdk::level::Level; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_pb::hummock::compact_task; - pub use risingwave_pb::hummock::{Level, LevelType}; + pub use risingwave_pb::hummock::LevelType; use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; @@ -213,7 +214,7 @@ mod test { ), Level { level_idx: 4, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_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), @@ -232,8 +233,8 @@ mod test { { let sst_10 = levels[3].table_infos.get_mut(8).unwrap(); - assert_eq!(10, sst_10.get_sst_id()); - sst_10.key_range.as_mut().unwrap().right_exclusive = true; + assert_eq!(10, sst_10.sst_id); + sst_10.key_range.right_exclusive = true; } assert_eq!(levels.len(), 4); @@ -294,7 +295,7 @@ mod test { let mut start_id = 2; for sst in &task.input.input_levels[0].table_infos { - assert_eq!(start_id, sst.get_sst_id()); + assert_eq!(start_id, sst.sst_id); start_id += 1; } @@ -344,7 +345,7 @@ mod test { )); let mut start_id = 8; for sst in &task.input.input_levels[0].table_infos { - assert_eq!(start_id, sst.get_sst_id()); + assert_eq!(start_id, sst.sst_id); start_id += 1; } @@ -476,7 +477,7 @@ mod test { let select_sst = &task.input.input_levels[0] .table_infos .iter() - .map(|sst| sst.get_sst_id()) + .map(|sst| sst.sst_id) .collect_vec(); assert!(select_sst.is_sorted()); assert_eq!(expect_task_sst_id_range[index], *select_sst); @@ -537,7 +538,7 @@ mod test { let select_sst = &task.input.input_levels[0] .table_infos .iter() - .map(|sst| sst.get_sst_id()) + .map(|sst| sst.sst_id) .collect_vec(); assert!(select_sst.is_sorted()); assert_eq!(expect_task_sst_id_range[index], *select_sst); 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 ea25211b4474..776a83019dc6 100644 --- a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs @@ -14,8 +14,8 @@ use std::sync::Arc; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{CompactionConfig, InputLevel, LevelType, OverlappingLevel}; +use risingwave_hummock_sdk::level::{InputLevel, Levels, OverlappingLevel}; +use risingwave_pb::hummock::{CompactionConfig, LevelType}; use super::{ CompactionInput, CompactionPicker, CompactionTaskValidator, LocalPickerStatistic, @@ -55,7 +55,7 @@ impl TierCompactionPicker { stats: &mut LocalPickerStatistic, ) -> Option { for (idx, level) in l0.sub_levels.iter().enumerate() { - if level.level_type() != LevelType::Overlapping { + if level.level_type != LevelType::Overlapping { continue; } @@ -165,8 +165,8 @@ pub mod tests { use std::sync::Arc; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::new_sub_level; - use risingwave_pb::hummock::hummock_version::Levels; - use risingwave_pb::hummock::{LevelType, OverlappingLevel}; + use risingwave_hummock_sdk::level::{Levels, OverlappingLevel}; + use risingwave_pb::hummock::LevelType; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::compaction::picker::{ 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 c7be0347fce6..0b3aad7ef8b2 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 @@ -14,8 +14,7 @@ use std::sync::Arc; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::InputLevel; +use risingwave_hummock_sdk::level::{InputLevel, Levels}; use crate::hummock::compaction::overlap_strategy::OverlapStrategy; use crate::hummock::compaction::picker::CompactionInput; @@ -133,7 +132,7 @@ impl TombstoneReclaimCompactionPicker { #[cfg(test)] pub mod tests { - use risingwave_pb::hummock::OverlappingLevel; + use risingwave_hummock_sdk::level::OverlappingLevel; use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; 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 bab0dca0c324..96fcc69175bd 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 @@ -14,7 +14,9 @@ use std::sync::Arc; -use risingwave_pb::hummock::{InputLevel, LevelType, SstableInfo}; +use risingwave_hummock_sdk::level::InputLevel; +use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_pb::hummock::LevelType; use super::{CompactionInput, LocalPickerStatistic}; use crate::hummock::compaction::overlap_strategy::OverlapStrategy; @@ -84,12 +86,12 @@ impl TrivialMovePicker { input_levels: vec![ InputLevel { level_idx: self.level as u32, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![trivial_move_sst], }, InputLevel { level_idx: self.target_level as u32, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![], }, ], 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 9efbfcd02076..51a1d56f34dc 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 @@ -14,12 +14,13 @@ use std::collections::{HashMap, HashSet}; +use bytes::Bytes; use risingwave_common::catalog::TableOption; use risingwave_common::util::epoch::Epoch; use risingwave_hummock_sdk::compaction_group::StateTableId; -use risingwave_hummock_sdk::key_range::KeyRangeCommon; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{InputLevel, KeyRange, SstableInfo}; +use risingwave_hummock_sdk::key_range::{KeyRange, KeyRangeCommon}; +use risingwave_hummock_sdk::level::{InputLevel, Levels}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use super::CompactionInput; use crate::hummock::level_handler::LevelHandler; @@ -46,7 +47,7 @@ impl TtlPickerState { pub fn init(&mut self, key_range: KeyRange) { self.last_select_end_bound = KeyRange { - left: vec![], + left: Bytes::default(), right: key_range.left.clone(), right_exclusive: true, }; @@ -137,9 +138,9 @@ impl TtlReclaimCompactionPicker { let last_sst = reclaimed_level.table_infos.last().unwrap(); let key_range_this_round = KeyRange { - left: first_sst.key_range.as_ref().unwrap().left.clone(), - right: last_sst.key_range.as_ref().unwrap().right.clone(), - right_exclusive: last_sst.key_range.as_ref().unwrap().right_exclusive, + left: first_sst.key_range.left.clone(), + right: last_sst.key_range.right.clone(), + right_exclusive: last_sst.key_range.right_exclusive, }; state.init(key_range_this_round); @@ -148,11 +149,7 @@ impl TtlReclaimCompactionPicker { let current_epoch_physical_time = Epoch::now().physical_time(); for sst in &reclaimed_level.table_infos { - let unmatched_sst = sst - .key_range - .as_ref() - .unwrap() - .sstable_overlap(&state.last_select_end_bound); + let unmatched_sst = sst.key_range.sstable_overlap(&state.last_select_end_bound); if unmatched_sst || level_handler.is_pending_compact(&sst.sst_id) @@ -173,9 +170,9 @@ impl TtlReclaimCompactionPicker { let select_last_sst = select_input_ssts.last().unwrap(); state.last_select_end_bound.full_key_extend(&KeyRange { - left: vec![], - right: select_last_sst.key_range.as_ref().unwrap().right.clone(), - right_exclusive: select_last_sst.key_range.as_ref().unwrap().right_exclusive, + left: Bytes::default(), + right: select_last_sst.key_range.right.clone(), + right_exclusive: select_last_sst.key_range.right_exclusive, }); Some(CompactionInput { @@ -205,9 +202,10 @@ mod test { use std::sync::Arc; use itertools::Itertools; + use risingwave_hummock_sdk::level::Level; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_pb::hummock::compact_task; - pub use risingwave_pb::hummock::{Level, LevelType}; + pub use risingwave_pb::hummock::LevelType; use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; @@ -247,7 +245,7 @@ mod test { ), Level { level_idx: 4, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![ generate_table_with_ids_and_epochs(2, 1, 0, 100, 1, vec![2], expired_epoch, 0), generate_table_with_ids_and_epochs( @@ -350,8 +348,8 @@ mod test { { let sst_10 = levels[3].table_infos.get_mut(8).unwrap(); - assert_eq!(10, sst_10.get_sst_id()); - sst_10.key_range.as_mut().unwrap().right_exclusive = true; + assert_eq!(10, sst_10.sst_id); + sst_10.key_range.right_exclusive = true; } assert_eq!(levels.len(), 4); @@ -398,7 +396,7 @@ mod test { let mut start_id = 2; for sst in &task.input.input_levels[0].table_infos { - assert_eq!(start_id, sst.get_sst_id()); + assert_eq!(start_id, sst.sst_id); start_id += 1; } @@ -455,7 +453,7 @@ mod test { let mut start_id = 3; for sst in &task.input.input_levels[0].table_infos { - assert_eq!(start_id, sst.get_sst_id()); + assert_eq!(start_id, sst.sst_id); start_id += 1; } @@ -496,7 +494,7 @@ mod test { compact_task::TaskType::Ttl )); for sst in &task.input.input_levels[0].table_infos { - assert_eq!(start_id, sst.get_sst_id()); + assert_eq!(start_id, sst.sst_id); start_id += 1; } } @@ -552,7 +550,7 @@ mod test { // 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!(select_sst.get_sst_id(), 5); + 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); @@ -668,7 +666,7 @@ mod test { let select_sst = &task.input.input_levels[0] .table_infos .iter() - .map(|sst| sst.get_sst_id()) + .map(|sst| sst.sst_id) .collect_vec(); assert!(select_sst.is_sorted()); assert_eq!(expect_task_sst_id_range[index], *select_sst); @@ -765,7 +763,7 @@ mod test { let select_sst = &task.input.input_levels[0] .table_infos .iter() - .map(|sst| sst.get_sst_id()) + .map(|sst| sst.sst_id) .collect_vec(); assert!(select_sst.is_sorted()); assert_eq!(expect_task_sst_id_range[index], *select_sst); 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 f0344d63b663..50a33c7d42e4 100644 --- a/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs +++ b/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs @@ -16,9 +16,9 @@ use std::collections::BTreeMap; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::key::{FullKey, TableKey}; +use risingwave_hummock_sdk::level::{InputLevel, Levels}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_watermark::ReadTableWatermark; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{InputLevel, SstableInfo}; use crate::hummock::compaction::picker::CompactionInput; use crate::hummock::level_handler::LevelHandler; @@ -77,8 +77,8 @@ fn should_delete_sst_by_watermark( ) -> bool { // Both table id and vnode must be identical for both the left and right keys in a SST. // As more data is written to the bottommost level, they will eventually become identical. - let left_key = FullKey::decode(&sst_info.key_range.as_ref().unwrap().left); - let right_key = FullKey::decode(&sst_info.key_range.as_ref().unwrap().right); + let left_key = FullKey::decode(&sst_info.key_range.left); + let right_key = FullKey::decode(&sst_info.key_range.right); if left_key.user_key.table_id != right_key.user_key.table_id { return false; } @@ -108,8 +108,9 @@ mod tests { use bytes::{BufMut, Bytes, BytesMut}; use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::key::{FullKey, TableKey}; + use risingwave_hummock_sdk::key_range::KeyRange; + use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_watermark::{ReadTableWatermark, WatermarkDirection}; - use risingwave_pb::hummock::{KeyRange, SstableInfo}; use crate::hummock::compaction::picker::vnode_watermark_picker::should_delete_sst_by_watermark; @@ -134,11 +135,15 @@ mod tests { let sst_info = SstableInfo { object_id: 1, sst_id: 1, - key_range: Some(KeyRange { - left: FullKey::new(2.into(), table_key(16, "some_watermark_key_1"), 0).encode(), - right: FullKey::new(2.into(), table_key(16, "some_watermark_key_2"), 0).encode(), + key_range: KeyRange { + left: FullKey::new(2.into(), table_key(16, "some_watermark_key_1"), 0) + .encode() + .into(), + right: FullKey::new(2.into(), table_key(16, "some_watermark_key_2"), 0) + .encode() + .into(), right_exclusive: true, - }), + }, table_ids: vec![2], ..Default::default() }; @@ -150,11 +155,15 @@ mod tests { let sst_info = SstableInfo { object_id: 1, sst_id: 1, - key_range: Some(KeyRange { - left: FullKey::new(1.into(), table_key(13, "some_watermark_key_1"), 0).encode(), - right: FullKey::new(1.into(), table_key(14, "some_watermark_key_2"), 0).encode(), + key_range: KeyRange { + left: FullKey::new(1.into(), table_key(13, "some_watermark_key_1"), 0) + .encode() + .into(), + right: FullKey::new(1.into(), table_key(14, "some_watermark_key_2"), 0) + .encode() + .into(), right_exclusive: true, - }), + }, table_ids: vec![1], ..Default::default() }; @@ -166,11 +175,15 @@ mod tests { let sst_info = SstableInfo { object_id: 1, sst_id: 1, - key_range: Some(KeyRange { - left: FullKey::new(1.into(), table_key(16, "some_watermark_key_1"), 0).encode(), - right: FullKey::new(1.into(), table_key(17, "some_watermark_key_2"), 0).encode(), + key_range: KeyRange { + left: FullKey::new(1.into(), table_key(16, "some_watermark_key_1"), 0) + .encode() + .into(), + right: FullKey::new(1.into(), table_key(17, "some_watermark_key_2"), 0) + .encode() + .into(), right_exclusive: true, - }), + }, table_ids: vec![1], ..Default::default() }; @@ -182,11 +195,15 @@ mod tests { let sst_info = SstableInfo { object_id: 1, sst_id: 1, - key_range: Some(KeyRange { - left: FullKey::new(1.into(), table_key(16, "some_watermark_key_1"), 0).encode(), - right: FullKey::new(1.into(), table_key(16, "some_watermark_key_9"), 0).encode(), + key_range: KeyRange { + left: FullKey::new(1.into(), table_key(16, "some_watermark_key_1"), 0) + .encode() + .into(), + right: FullKey::new(1.into(), table_key(16, "some_watermark_key_9"), 0) + .encode() + .into(), right_exclusive: true, - }), + }, table_ids: vec![1], ..Default::default() }; @@ -198,11 +215,15 @@ mod tests { let sst_info = SstableInfo { object_id: 1, sst_id: 1, - key_range: Some(KeyRange { - left: FullKey::new(1.into(), table_key(16, "some_watermark_key_1"), 0).encode(), - right: FullKey::new(1.into(), table_key(16, "some_watermark_key_2"), 0).encode(), + key_range: KeyRange { + left: FullKey::new(1.into(), table_key(16, "some_watermark_key_1"), 0) + .encode() + .into(), + right: FullKey::new(1.into(), table_key(16, "some_watermark_key_2"), 0) + .encode() + .into(), right_exclusive: true, - }), + }, table_ids: vec![1], ..Default::default() }; diff --git a/src/meta/src/hummock/compaction/selector/level_selector.rs b/src/meta/src/hummock/compaction/selector/level_selector.rs index b48455b49d4c..918361bc4d44 100644 --- a/src/meta/src/hummock/compaction/selector/level_selector.rs +++ b/src/meta/src/hummock/compaction/selector/level_selector.rs @@ -18,10 +18,10 @@ // (found in the LICENSE.Apache file in the root directory). use std::sync::Arc; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockLevelsExt; +use risingwave_hummock_sdk::level::Levels; use risingwave_hummock_sdk::HummockCompactionTaskId; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{compact_task, CompactionConfig, LevelType}; +use risingwave_pb::hummock::compact_task::PbTaskType; +use risingwave_pb::hummock::{CompactionConfig, LevelType}; use super::{ create_compaction_task, CompactionSelector, LevelCompactionPicker, TierCompactionPicker, @@ -227,7 +227,7 @@ impl DynamicLevelSelectorCore { .unwrap() .sub_levels .iter() - .filter(|level| level.level_type() == LevelType::Overlapping) + .filter(|level| level.level_type == LevelType::Overlapping) .map(|level| level.table_infos.len()) .sum::(); if overlapping_file_count > 0 { @@ -256,7 +256,7 @@ impl DynamicLevelSelectorCore { .iter() .filter(|level| { level.vnode_partition_count == self.config.split_weight_by_vnode - && level.level_type() == LevelType::Nonoverlapping + && level.level_type == LevelType::Nonoverlapping }) .map(|level| level.total_file_size) .sum::() @@ -274,7 +274,7 @@ impl DynamicLevelSelectorCore { .unwrap() .sub_levels .iter() - .filter(|level| level.level_type() == LevelType::Nonoverlapping) + .filter(|level| level.level_type == LevelType::Nonoverlapping) .count() as u64; let non_overlapping_level_score = non_overlapping_level_count * SCORE_BASE / std::cmp::max( @@ -375,7 +375,7 @@ impl DynamicLevelSelectorCore { let mut level_bytes; let mut next_level_bytes = 0; for level in &levels.levels[ctx.base_level - 1..levels.levels.len()] { - let level_index = level.get_level_idx() as usize; + let level_index = level.level_idx as usize; if next_level_bytes > 0 { level_bytes = next_level_bytes; @@ -472,8 +472,8 @@ impl CompactionSelector for DynamicLevelSelector { "DynamicLevelSelector" } - fn task_type(&self) -> compact_task::TaskType { - compact_task::TaskType::Dynamic + fn task_type(&self) -> PbTaskType { + PbTaskType::Dynamic } } @@ -484,9 +484,9 @@ pub mod tests { use itertools::Itertools; use risingwave_common::constants::hummock::CompactionFilterFlag; + use risingwave_hummock_sdk::level::Levels; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_pb::hummock::compaction_config::CompactionMode; - use risingwave_pb::hummock::hummock_version::Levels; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::compaction::selector::tests::{ @@ -688,7 +688,7 @@ pub mod tests { compaction.input.input_levels[0] .table_infos .iter() - .map(|sst| sst.get_sst_id()) + .map(|sst| sst.sst_id) .collect_vec(), vec![5] ); @@ -696,7 +696,7 @@ pub mod tests { compaction.input.input_levels[1] .table_infos .iter() - .map(|sst| sst.get_sst_id()) + .map(|sst| sst.sst_id) .collect_vec(), vec![10] ); diff --git a/src/meta/src/hummock/compaction/selector/manual_selector.rs b/src/meta/src/hummock/compaction/selector/manual_selector.rs index 8c814a69b03d..82f9cbfc3e25 100644 --- a/src/meta/src/hummock/compaction/selector/manual_selector.rs +++ b/src/meta/src/hummock/compaction/selector/manual_selector.rs @@ -19,9 +19,11 @@ use std::collections::HashSet; +use bytes::Bytes; use risingwave_hummock_sdk::compaction_group::StateTableId; +use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::{HummockCompactionTaskId, HummockSstableId}; -use risingwave_pb::hummock::{compact_task, KeyRange}; +use risingwave_pb::hummock::compact_task; use super::{CompactionSelector, DynamicLevelSelectorCore}; use crate::hummock::compaction::picker::{ @@ -47,8 +49,8 @@ impl Default for ManualCompactionOption { Self { sst_ids: vec![], key_range: KeyRange { - left: vec![], - right: vec![], + left: Bytes::default(), + right: Bytes::default(), right_exclusive: false, }, internal_table_id: HashSet::default(), diff --git a/src/meta/src/hummock/compaction/selector/mod.rs b/src/meta/src/hummock/compaction/selector/mod.rs index b685cf381267..86571bf1f65b 100644 --- a/src/meta/src/hummock/compaction/selector/mod.rs +++ b/src/meta/src/hummock/compaction/selector/mod.rs @@ -32,11 +32,11 @@ pub use emergency_selector::EmergencySelector; pub use level_selector::{DynamicLevelSelector, DynamicLevelSelectorCore}; pub use manual_selector::{ManualCompactionOption, ManualCompactionSelector}; use risingwave_common::catalog::{TableId, TableOption}; +use risingwave_hummock_sdk::level::Levels; use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_hummock_sdk::HummockCompactionTaskId; use risingwave_pb::hummock::compact_task; -use risingwave_pb::hummock::hummock_version::Levels; pub use space_reclaim_selector::SpaceReclaimCompactionSelector; pub use tombstone_compaction_selector::TombstoneCompactionSelector; pub use ttl_selector::TtlCompactionSelector; @@ -127,7 +127,10 @@ pub mod tests { use std::ops::Range; use itertools::Itertools; - use risingwave_pb::hummock::{KeyRange, Level, LevelType, OverlappingLevel, SstableInfo}; + use risingwave_hummock_sdk::key_range::KeyRange; + use risingwave_hummock_sdk::level::{Level, OverlappingLevel}; + use risingwave_hummock_sdk::sstable_info::SstableInfo; + use risingwave_pb::hummock::LevelType; use super::*; use crate::hummock::test_utils::iterator_test_key_of_epoch; @@ -136,10 +139,10 @@ pub mod tests { levels.l0.as_mut().unwrap().total_file_size += sst.file_size; levels.l0.as_mut().unwrap().sub_levels.push(Level { level_idx: 0, - level_type: LevelType::Overlapping as i32, + level_type: LevelType::Overlapping, total_file_size: sst.file_size, uncompressed_file_size: sst.uncompressed_file_size, - sub_level_id: sst.get_sst_id(), + sub_level_id: sst.sst_id, table_infos: vec![sst], ..Default::default() }); @@ -154,7 +157,7 @@ pub mod tests { .sub_levels .last_mut() .unwrap() - .level_type = LevelType::Nonoverlapping as i32; + .level_type = LevelType::Nonoverlapping; } pub fn push_tables_level0_nonoverlapping(levels: &mut Levels, table_infos: Vec) { @@ -163,11 +166,11 @@ pub mod tests { .iter() .map(|table| table.uncompressed_file_size) .sum(); - let sub_level_id = table_infos[0].get_sst_id(); + let sub_level_id = table_infos[0].sst_id; levels.l0.as_mut().unwrap().total_file_size += total_file_size; levels.l0.as_mut().unwrap().sub_levels.push(Level { level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, total_file_size, sub_level_id, table_infos, @@ -186,11 +189,11 @@ pub mod tests { SstableInfo { object_id: id, sst_id: id, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(table_prefix, left, epoch), - right: iterator_test_key_of_epoch(table_prefix, right, epoch), + key_range: KeyRange { + left: iterator_test_key_of_epoch(table_prefix, left, epoch).into(), + right: iterator_test_key_of_epoch(table_prefix, right, epoch).into(), right_exclusive: false, - }), + }, file_size: (right - left + 1) as u64, table_ids: vec![table_prefix as u32], uncompressed_file_size: (right - left + 1) as u64, @@ -213,11 +216,11 @@ pub mod tests { SstableInfo { object_id: id, sst_id: id, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(table_prefix, left, epoch), - right: iterator_test_key_of_epoch(table_prefix, right, epoch), + key_range: KeyRange { + left: iterator_test_key_of_epoch(table_prefix, left, epoch).into(), + right: iterator_test_key_of_epoch(table_prefix, right, epoch).into(), right_exclusive: false, - }), + }, file_size: (right - left + 1) as u64, table_ids, uncompressed_file_size: (right - left + 1) as u64, @@ -253,7 +256,7 @@ pub mod tests { .sum(); Level { level_idx, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos, total_file_size, sub_level_id: 0, @@ -276,7 +279,7 @@ pub mod tests { .enumerate() .map(|(idx, table)| Level { level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, total_file_size: table.file_size, uncompressed_file_size: table.uncompressed_file_size, sub_level_id: idx as u64, @@ -298,7 +301,7 @@ pub mod tests { .enumerate() .map(|(idx, table)| Level { level_idx: 0, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, total_file_size: table.iter().map(|table| table.file_size).sum::(), uncompressed_file_size: table .iter() @@ -333,7 +336,7 @@ pub mod tests { .enumerate() .map(|(idx, table)| Level { level_idx: 0, - level_type: LevelType::Overlapping as i32, + level_type: LevelType::Overlapping, total_file_size: table.iter().map(|table| table.file_size).sum::(), sub_level_id: idx as u64, table_infos: table.clone(), diff --git a/src/meta/src/hummock/compactor_manager.rs b/src/meta/src/hummock/compactor_manager.rs index ab0c868f703c..252f92c40401 100644 --- a/src/meta/src/hummock/compactor_manager.rs +++ b/src/meta/src/hummock/compactor_manager.rs @@ -19,11 +19,11 @@ use std::time::{Duration, Instant, SystemTime}; use fail::fail_point; use parking_lot::RwLock; use risingwave_hummock_sdk::compact::statistics_compact_task; +use risingwave_hummock_sdk::compact_task::CompactTask; use risingwave_hummock_sdk::{HummockCompactionTaskId, HummockContextId}; use risingwave_pb::hummock::subscribe_compaction_event_response::Event as ResponseEvent; use risingwave_pb::hummock::{ - CancelCompactTask, CompactTask, CompactTaskAssignment, CompactTaskProgress, - SubscribeCompactionEventResponse, + CancelCompactTask, CompactTaskAssignment, CompactTaskProgress, SubscribeCompactionEventResponse, }; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; @@ -156,7 +156,7 @@ impl CompactorManagerInner { }; // Initialize heartbeat for existing tasks. task_assignment.into_iter().for_each(|assignment| { - manager.initiate_task_heartbeat(assignment.compact_task.unwrap()); + manager.initiate_task_heartbeat(CompactTask::from(assignment.compact_task.unwrap())); }); Ok(manager) } @@ -302,7 +302,7 @@ impl CompactorManagerInner { task.target_level, task.base_level, task.target_sub_level_id, - task.task_type, + task.task_type.as_str_name(), compact_task_statistics ); } diff --git a/src/meta/src/hummock/level_handler.rs b/src/meta/src/hummock/level_handler.rs index 25bc8af8e66d..2f729b7123d5 100644 --- a/src/meta/src/hummock/level_handler.rs +++ b/src/meta/src/hummock/level_handler.rs @@ -15,9 +15,10 @@ use std::collections::HashMap; use itertools::Itertools; +use risingwave_hummock_sdk::level::Level; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::{HummockCompactionTaskId, HummockSstableId}; use risingwave_pb::hummock::level_handler::RunningCompactTask; -use risingwave_pb::hummock::{Level, SstableInfo}; #[derive(Clone, Debug, PartialEq)] pub struct LevelHandler { @@ -90,9 +91,9 @@ impl LevelHandler { let mut table_ids = vec![]; let mut total_file_size = 0; for sst in ssts { - self.compacting_files.insert(sst.get_sst_id(), task_id); + self.compacting_files.insert(sst.sst_id, task_id); total_file_size += sst.file_size; - table_ids.push(sst.get_sst_id()); + table_ids.push(sst.sst_id); } self.pending_tasks.push(RunningCompactTask { diff --git a/src/meta/src/hummock/manager/checkpoint.rs b/src/meta/src/hummock/manager/checkpoint.rs index 676b2ada1557..e901c09f7831 100644 --- a/src/meta/src/hummock/manager/checkpoint.rs +++ b/src/meta/src/hummock/manager/checkpoint.rs @@ -22,10 +22,10 @@ use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ }; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::HummockVersionId; -use risingwave_pb::hummock::hummock_version_checkpoint::{ - StaleObjects as PbStaleObjects, StaleObjects, +use risingwave_pb::hummock::hummock_version_checkpoint::{PbStaleObjects, StaleObjects}; +use risingwave_pb::hummock::{ + PbHummockVersion, PbHummockVersionArchive, PbHummockVersionCheckpoint, }; -use risingwave_pb::hummock::{PbHummockVersionArchive, PbHummockVersionCheckpoint}; use thiserror_ext::AsReport; use tracing::warn; @@ -60,7 +60,7 @@ impl HummockVersionCheckpoint { pub fn to_protobuf(&self) -> PbHummockVersionCheckpoint { PbHummockVersionCheckpoint { - version: Some(self.version.to_protobuf()), + version: Some(PbHummockVersion::from(&self.version)), stale_objects: self.stale_objects.clone(), } } @@ -197,11 +197,11 @@ impl HummockManager { ); if self.env.opts.enable_hummock_data_archive { archive = Some(PbHummockVersionArchive { - version: Some(old_checkpoint.version.to_protobuf()), + version: Some(PbHummockVersion::from(&old_checkpoint.version)), version_deltas: versioning .hummock_version_deltas .range((Excluded(old_checkpoint_id), Included(new_checkpoint_id))) - .map(|(_, version_delta)| version_delta.to_protobuf()) + .map(|(_, version_delta)| version_delta.into()) .collect(), }); } diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index cae89ab34d9d..2e8535e97de7 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -15,8 +15,10 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::change_log::ChangeLogDelta; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::split_sst; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::{ add_prost_table_stats_map, purge_prost_table_stats, to_prost_table_stats_map, PbTableStatsMap, }; @@ -26,8 +28,7 @@ use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockEpoch, HummockSstableObjectId, LocalSstableInfo, }; use risingwave_pb::hummock::compact_task::{self}; -use risingwave_pb::hummock::hummock_version_delta::ChangeLogDelta; -use risingwave_pb::hummock::{HummockSnapshot, SstableInfo}; +use risingwave_pb::hummock::HummockSnapshot; use sea_orm::TransactionTrait; use crate::hummock::error::{Error, Result}; @@ -349,7 +350,7 @@ impl HummockManager { let mut sst_to_cg_vec = Vec::with_capacity(sstables.len()); for commit_sst in sstables { let mut group_table_ids: BTreeMap> = BTreeMap::new(); - for table_id in commit_sst.sst_info.get_table_ids() { + for table_id in &commit_sst.sst_info.table_ids { match table_compaction_group_mapping.get(&TableId::new(*table_id)) { Some(cg_id_from_meta) => { group_table_ids @@ -361,7 +362,7 @@ impl HummockManager { tracing::warn!( "table {} in SST {} doesn't belong to any compaction group", table_id, - commit_sst.sst_info.get_object_id(), + commit_sst.sst_info.object_id, ); } } diff --git a/src/meta/src/hummock/manager/compaction.rs b/src/meta/src/hummock/manager/compaction.rs index d8d2d709189c..fbe0f1fe8a66 100644 --- a/src/meta/src/hummock/manager/compaction.rs +++ b/src/meta/src/hummock/manager/compaction.rs @@ -41,29 +41,31 @@ use parking_lot::Mutex; use rand::seq::SliceRandom; use rand::thread_rng; use risingwave_common::util::epoch::Epoch; +use risingwave_hummock_sdk::compact_task::{CompactTask, ReportTask}; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockLevelsExt; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::level::{InputLevel, Level, Levels}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::{ add_prost_table_stats_map, purge_prost_table_stats, PbTableStatsMap, }; -use risingwave_hummock_sdk::version::HummockVersion; +use risingwave_hummock_sdk::version::{GroupDelta, HummockVersion, IntraLevelDelta}; use risingwave_hummock_sdk::{ compact_task_to_string, statistics_compact_task, CompactionGroupId, HummockCompactionTaskId, HummockVersionId, }; use risingwave_pb::hummock::compact_task::{TaskStatus, TaskType}; -use risingwave_pb::hummock::group_delta::DeltaType; -use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::subscribe_compaction_event_request::{ - self, Event as RequestEvent, HeartBeat, PullTask, ReportTask, + self, Event as RequestEvent, HeartBeat, PullTask, }; use risingwave_pb::hummock::subscribe_compaction_event_response::{ Event as ResponseEvent, PullTaskAck, }; use risingwave_pb::hummock::{ - compact_task, CompactStatus as PbCompactStatus, CompactTask, CompactTaskAssignment, - CompactionConfig, GroupDelta, InputLevel, IntraLevelDelta, Level, SstableInfo, - StateTableInfoDelta, SubscribeCompactionEventRequest, TableOption, TableSchema, + compact_task, CompactTaskAssignment, CompactionConfig, PbCompactStatus, + PbCompactTaskAssignment, StateTableInfoDelta, SubscribeCompactionEventRequest, TableOption, + TableSchema, }; use rw_futures_util::pending_on_none; use thiserror_ext::AsReport; @@ -155,11 +157,8 @@ 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.get_sst_id()) - .collect_vec(); + let mut removed_table_ids = + level.table_infos.iter().map(|sst| sst.sst_id).collect_vec(); removed_table_ids_map .entry(level_idx) @@ -168,25 +167,25 @@ impl<'a> HummockVersionTransaction<'a> { } for (level_idx, removed_table_ids) in removed_table_ids_map { - let group_delta = GroupDelta { - delta_type: Some(DeltaType::IntraLevel(IntraLevelDelta { - level_idx, - removed_table_ids, - ..Default::default() - })), - }; + let group_delta = GroupDelta::IntraLevel(IntraLevelDelta::new( + level_idx, + 0, // default + removed_table_ids, + vec![], // default + 0, // default + )); + group_deltas.push(group_delta); } - let group_delta = GroupDelta { - delta_type: Some(DeltaType::IntraLevel(IntraLevelDelta { - level_idx: compact_task.target_level, - inserted_table_infos: compact_task.sorted_output_ssts.clone(), - l0_sub_level_id: compact_task.target_sub_level_id, - vnode_partition_count: compact_task.split_weight_by_vnode, - ..Default::default() - })), - }; + let group_delta = GroupDelta::IntraLevel(IntraLevelDelta::new( + compact_task.target_level, + compact_task.target_sub_level_id, + vec![], // default + compact_task.sorted_output_ssts.clone(), + compact_task.split_weight_by_vnode, + )); + group_deltas.push(group_delta); let new_visible_table_safe_epoch = std::cmp::max( version_delta.latest_version().visible_table_safe_epoch(), @@ -227,7 +226,7 @@ impl<'a> HummockVersionTransaction<'a> { #[derive(Default)] pub struct Compaction { /// Compaction task that is already assigned to a compactor - pub compact_task_assignment: BTreeMap, + pub compact_task_assignment: BTreeMap, /// `CompactStatus` of each compaction group pub compaction_statuses: BTreeMap, @@ -325,7 +324,7 @@ impl HummockManager { for task in compact_tasks { let task_id = task.task_id; if let Err(e) = - compactor.send_event(ResponseEvent::CompactTask(task)) + compactor.send_event(ResponseEvent::CompactTask(task.into())) { tracing::warn!( error = %e.as_report(), @@ -603,7 +602,7 @@ impl HummockManager { } RequestEvent::ReportTask(task) => { - report_events.push(task); + report_events.push(task.into()); } _ => unreachable!(), @@ -621,7 +620,7 @@ impl HummockManager { } RequestEvent::ReportTask(task) => { - report_events.push(task); + report_events.push(task.into()); if report_events.len() >= MAX_REPORT_COUNT { break; } @@ -783,11 +782,9 @@ impl HummockManager { _ => 0, }; let vnode_partition_count = compact_task.input.vnode_partition_count; - use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; - let mut compact_task = CompactTask { input_ssts: compact_task.input.input_levels, - splits: vec![risingwave_pb::hummock::KeyRange::inf()], + splits: vec![KeyRange::inf()], watermark, sorted_output_ssts: vec![], task_id, @@ -799,7 +796,7 @@ impl HummockManager { .get_compaction_group_levels(compaction_group_id) .is_last_level(target_level_id), base_level: compact_task.base_level as u32, - task_status: TaskStatus::Pending as i32, + task_status: TaskStatus::Pending, compaction_group_id: group_config.group_id, existing_table_ids: member_table_ids.clone(), compression_algorithm, @@ -813,7 +810,7 @@ impl HummockManager { current_epoch_time: Epoch::now().0, compaction_filter_mask: group_config.compaction_config.compaction_filter_mask, target_sub_level_id: compact_task.input.target_sub_level_id, - task_type: compact_task.compaction_task_type as i32, + task_type: compact_task.compaction_task_type, split_weight_by_vnode: vnode_partition_count, max_sub_compaction: group_config.compaction_config.max_sub_compaction, ..Default::default() @@ -840,7 +837,7 @@ impl HummockManager { compact_task.input_ssts, start_time.elapsed() ); - compact_task.set_task_status(TaskStatus::Success); + compact_task.task_status = TaskStatus::Success; compact_status.report_compact_task(&compact_task); if !is_trivial_reclaim { compact_task @@ -894,7 +891,7 @@ impl HummockManager { compact_task_assignment.insert( compact_task.task_id, CompactTaskAssignment { - compact_task: Some(compact_task.clone()), + compact_task: Some(compact_task.clone().into()), context_id: META_NODE_ID, // deprecated }, ); @@ -954,7 +951,7 @@ impl HummockManager { .initiate_task_heartbeat(compact_task.clone()); // this task has been finished. - compact_task.set_task_status(TaskStatus::Pending); + compact_task.task_status = TaskStatus::Pending; let compact_task_statistics = statistics_compact_task(compact_task); let level_type_label = build_compact_task_level_type_metrics_label( @@ -992,7 +989,7 @@ impl HummockManager { "For compaction group {}: pick up {} {} sub_level in level {} to compact to target {}. cost time: {:?} compact_task_statistics {:?}", compaction_group_id, level_count, - compact_task.input_ssts[0].level_type().as_str_name(), + compact_task.input_ssts[0].level_type.as_str_name(), compact_task.input_ssts[0].level_idx, compact_task.target_level, start_time.elapsed(), @@ -1037,7 +1034,7 @@ impl HummockManager { .into_iter() .map(|task_id| ReportTask { task_id, - task_status: task_status as i32, + task_status, sorted_output_ssts: vec![], table_stats_change: HashMap::default(), }) @@ -1064,7 +1061,7 @@ impl HummockManager { .get_compact_tasks_impl(compaction_groups, max_select_count, selector) .await?; tasks.retain(|task| { - if task.task_status() == TaskStatus::Success { + if task.task_status == TaskStatus::Success { debug_assert!( CompactStatus::is_trivial_reclaim(task) || CompactStatus::is_trivial_move_task(task) @@ -1090,7 +1087,7 @@ impl HummockManager { .get_compact_tasks_impl(vec![compaction_group_id], 1, selector) .await?; for task in normal_tasks { - if task.task_status() != TaskStatus::Success { + if task.task_status != TaskStatus::Success { return Ok(Some(task)); } debug_assert!( @@ -1120,7 +1117,7 @@ impl HummockManager { .levels .get(&compact_task.compaction_group_id) { - for input_level in compact_task.get_input_ssts() { + for input_level in &compact_task.input_ssts { let input_level: &InputLevel = input_level; let mut sst_ids: HashSet<_> = input_level .table_infos @@ -1133,7 +1130,7 @@ impl HummockManager { } } if input_level.level_idx == 0 { - for level in &group.get_level0().sub_levels { + for level in &group.level0().sub_levels { filter_ssts(level, &mut sst_ids); } } else { @@ -1158,7 +1155,7 @@ impl HummockManager { let rets = self .report_compact_tasks(vec![ReportTask { task_id, - task_status: task_status as i32, + task_status, sorted_output_ssts, table_stats_change: table_stats_change.unwrap_or_default(), }]) @@ -1216,7 +1213,7 @@ impl HummockManager { for (idx, task) in report_tasks.into_iter().enumerate() { rets[idx] = true; let mut compact_task = match compact_task_assignment.remove(task.task_id) { - Some(compact_task) => compact_task.compact_task.unwrap(), + Some(compact_task) => CompactTask::from(compact_task.compact_task.unwrap()), None => { tracing::warn!("{}", format!("compact task {} not found", task.task_id)); rets[idx] = false; @@ -1235,7 +1232,7 @@ impl HummockManager { compact_status.report_compact_task(&compact_task); } None => { - compact_task.set_task_status(TaskStatus::InvalidGroupCanceled); + compact_task.task_status = TaskStatus::InvalidGroupCanceled; } } @@ -1249,12 +1246,12 @@ impl HummockManager { .iter() .map(|level| level.level_idx) .collect(); - let is_success = if let TaskStatus::Success = compact_task.task_status() { + let is_success = if let TaskStatus::Success = compact_task.task_status { // if member_table_ids changes, the data of sstable may stale. let is_expired = Self::is_compact_task_expired(&compact_task, version.latest_version()); if is_expired { - compact_task.set_task_status(TaskStatus::InputOutdatedCanceled); + compact_task.task_status = TaskStatus::InputOutdatedCanceled; false } else { let group = version @@ -1265,7 +1262,7 @@ impl HummockManager { let input_exist = group.check_deleted_sst_exist(&input_level_ids, input_sst_ids); if !input_exist { - compact_task.set_task_status(TaskStatus::InvalidGroupCanceled); + compact_task.task_status = TaskStatus::InvalidGroupCanceled; warn!( "The task may be expired because of group split, task:\n {:?}", compact_task_to_string(&compact_task) @@ -1317,9 +1314,9 @@ impl HummockManager { } let mut success_groups = vec![]; for compact_task in tasks { - let task_status = compact_task.task_status(); + let task_status = compact_task.task_status; let task_status_label = task_status.as_str_name(); - let task_type_label = compact_task.task_type().as_str_name(); + let task_type_label = compact_task.task_type.as_str_name(); self.compactor_manager .remove_task_heartbeat(compact_task.task_id); @@ -1350,8 +1347,8 @@ impl HummockManager { ); if !deterministic_mode - && (matches!(compact_task.task_type(), compact_task::TaskType::Dynamic) - || matches!(compact_task.task_type(), compact_task::TaskType::Emergency)) + && (matches!(compact_task.task_type, compact_task::TaskType::Dynamic) + || matches!(compact_task.task_type, compact_task::TaskType::Emergency)) { // only try send Dynamic compaction self.try_send_compaction_request( @@ -1445,7 +1442,7 @@ impl HummockManager { let compact_task_string = compact_task_to_string(&compact_task); // TODO: shall we need to cancel on meta ? compactor - .send_event(ResponseEvent::CompactTask(compact_task)) + .send_event(ResponseEvent::CompactTask(compact_task.into())) .with_context(|| { format!( "Failed to trigger compaction task for compaction_group {}", @@ -1672,7 +1669,7 @@ impl HummockManager { guard.compact_task_assignment.insert( task_id, CompactTaskAssignment { - compact_task: Some(task), + compact_task: Some(task.into()), context_id: 0, }, ); @@ -1682,7 +1679,7 @@ impl HummockManager { // So we pass the modified compact_task directly into the `report_compact_task_impl` self.report_compact_tasks(vec![ReportTask { task_id, - task_status: task_status as i32, + task_status, sorted_output_ssts, table_stats_change: table_stats_change.unwrap_or_default(), }]) diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index ae2611c70be0..61191722f800 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -18,21 +18,20 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::compact_task::ReportTask; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ get_compaction_group_ids, TableGroupInfo, }; use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; +use risingwave_hummock_sdk::version::{GroupDelta, GroupDeltas}; use risingwave_hummock_sdk::CompactionGroupId; use risingwave_meta_model_v2::compaction_config; use risingwave_pb::hummock::compact_task::TaskStatus; -use risingwave_pb::hummock::group_delta::DeltaType; -use risingwave_pb::hummock::hummock_version_delta::GroupDeltas; use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_config::MutableConfig; -use risingwave_pb::hummock::subscribe_compaction_event_request::ReportTask; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ - compact_task, CompactionConfig, CompactionGroupInfo, CompatibilityVersion, GroupConstruct, - GroupDelta, GroupDestroy, StateTableInfoDelta, + compact_task, CompactionConfig, CompactionGroupInfo, CompatibilityVersion, PbGroupConstruct, + PbGroupDestroy, PbStateTableInfoDelta, }; use tokio::sync::OnceCell; @@ -252,20 +251,20 @@ impl HummockManager { } }; - group_deltas.push(GroupDelta { - delta_type: Some(DeltaType::GroupConstruct(GroupConstruct { - group_config: Some(config), - group_id, - ..Default::default() - })), + let group_delta = GroupDelta::GroupConstruct(PbGroupConstruct { + group_config: Some(config), + group_id, + ..Default::default() }); + + group_deltas.push(group_delta); } } assert!(new_version_delta .state_table_info_delta .insert( TableId::new(*table_id), - StateTableInfoDelta { + PbStateTableInfoDelta { committed_epoch: epoch, safe_epoch: epoch, compaction_group_id: *raw_group_id, @@ -330,7 +329,7 @@ impl HummockManager { new_version_delta .latest_version() .get_compaction_group_levels(group_id) - .get_levels() + .levels .len(), )); } @@ -343,9 +342,9 @@ impl HummockManager { .entry(*group_id) .or_default() .group_deltas; - group_deltas.push(GroupDelta { - delta_type: Some(DeltaType::GroupDestroy(GroupDestroy {})), - }); + + let group_delta = GroupDelta::GroupDestroy(PbGroupDestroy {}); + group_deltas.push(group_delta); } for (group_id, max_level) in groups_to_remove { @@ -532,16 +531,14 @@ impl HummockManager { new_version_delta.group_deltas.insert( new_compaction_group_id, GroupDeltas { - group_deltas: vec![GroupDelta { - delta_type: Some(DeltaType::GroupConstruct(GroupConstruct { - group_config: Some(config.clone()), - group_id: new_compaction_group_id, - parent_group_id, - new_sst_start_id, - table_ids: vec![], - version: CompatibilityVersion::NoMemberTableIds as i32, - })), - }], + group_deltas: vec![GroupDelta::GroupConstruct(PbGroupConstruct { + group_config: Some(config.clone()), + group_id: new_compaction_group_id, + parent_group_id, + new_sst_start_id, + table_ids: vec![], + version: CompatibilityVersion::NoMemberTableIds as i32, + })], }, ); ((new_compaction_group_id, config), new_compaction_group_id) @@ -561,7 +558,7 @@ impl HummockManager { .state_table_info_delta .insert( table_id, - StateTableInfoDelta { + PbStateTableInfoDelta { committed_epoch: info.committed_epoch, safe_epoch: info.safe_epoch, compaction_group_id: new_compaction_group_id, @@ -584,12 +581,14 @@ impl HummockManager { let mut canceled_tasks = vec![]; for task_assignment in compaction_guard.compact_task_assignment.values() { if let Some(task) = task_assignment.compact_task.as_ref() { - let need_cancel = - HummockManager::is_compact_task_expired(task, &versioning.current_version); + let need_cancel = HummockManager::is_compact_task_expired( + &task.into(), + &versioning.current_version, + ); if need_cancel { canceled_tasks.push(ReportTask { task_id: task.task_id, - task_status: TaskStatus::ManualCanceled as i32, + task_status: TaskStatus::ManualCanceled, table_stats_change: HashMap::default(), sorted_output_ssts: vec![], }); diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index d281ea4f7e4c..6bfcd119b89d 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -250,7 +250,7 @@ impl HummockManager { .collect_vec(); if compactor .send_event(ResponseEvent::ValidationTask(ValidationTask { - sst_infos, + sst_infos: sst_infos.into_iter().map(|sst| sst.into()).collect_vec(), sst_id_to_worker_id: sst_to_context.clone(), epoch, })) diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index f1e4a4ba426d..97a99945bcf4 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -430,7 +430,7 @@ mod tests { let committed_object_ids = sst_infos .into_iter() .flatten() - .map(|s| s.get_object_id()) + .map(|s| s.object_id) .sorted() .collect_vec(); assert!(!committed_object_ids.is_empty()); diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index d4b577d808d4..3bb4d5152adb 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -33,8 +33,8 @@ use risingwave_meta_model_v2::{ hummock_version_delta, hummock_version_stats, }; use risingwave_pb::hummock::{ - CompactTaskAssignment, HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, - HummockVersionStats, PbCompactionGroupInfo, SubscribeCompactionEventRequest, + HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, HummockVersionStats, + PbCompactTaskAssignment, PbCompactionGroupInfo, SubscribeCompactionEventRequest, }; use risingwave_pb::meta::subscribe_response::Operation; use tokio::sync::mpsc::UnboundedSender; @@ -343,7 +343,7 @@ impl HummockManager { } compaction_guard.compact_task_assignment = match &meta_store { - MetaStoreImpl::Kv(meta_store) => CompactTaskAssignment::list(meta_store) + MetaStoreImpl::Kv(meta_store) => PbCompactTaskAssignment::list(meta_store) .await? .into_iter() .map(|assigned| (assigned.key().unwrap(), assigned)) @@ -353,7 +353,12 @@ impl HummockManager { .await .map_err(MetadataModelError::from)? .into_iter() - .map(|m| (m.id as HummockCompactionTaskId, m.into())) + .map(|m| { + ( + m.id as HummockCompactionTaskId, + PbCompactTaskAssignment::from(m), + ) + }) .collect(), }; diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 5e677248903d..71f0d117acb0 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -24,8 +24,11 @@ use prometheus::Registry; use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::{test_epoch, EpochExt, INVALID_EPOCH}; use risingwave_hummock_sdk::compact::compact_task_to_string; +use risingwave_hummock_sdk::compact_task::CompactTask; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::get_compaction_group_ssts; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::{to_prost_table_stats_map, TableStats, TableStatsMap}; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ @@ -34,10 +37,7 @@ use risingwave_hummock_sdk::{ }; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::hummock::compact_task::TaskStatus; -use risingwave_pb::hummock::{ - CompactTask, HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, KeyRange, - SstableInfo, -}; +use risingwave_pb::hummock::{HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot}; use risingwave_pb::meta::add_worker_node_request::Property; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; @@ -66,11 +66,11 @@ fn pin_snapshots_epoch(pin_snapshots: &[HummockPinnedSnapshot]) -> Vec { fn gen_sstable_info(sst_id: u64, idx: usize, table_ids: Vec) -> SstableInfo { SstableInfo { sst_id, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(1, idx, 1), - right: iterator_test_key_of_epoch(1, idx, 1), + key_range: KeyRange { + left: iterator_test_key_of_epoch(1, idx, 1).into(), + right: iterator_test_key_of_epoch(1, idx, 1).into(), right_exclusive: false, - }), + }, table_ids, object_id: sst_id, min_epoch: 20, @@ -228,15 +228,8 @@ async fn test_hummock_compaction_task() { .await .unwrap() .unwrap(); - assert_eq!( - compact_task - .get_input_ssts() - .first() - .unwrap() - .get_level_idx(), - 0 - ); - assert_eq!(compact_task.get_task_id(), 2); + assert_eq!(compact_task.input_ssts.first().unwrap().level_idx, 0); + assert_eq!(compact_task.task_id, 2); // Cancel the task and succeed. assert!(hummock_manager @@ -253,7 +246,7 @@ async fn test_hummock_compaction_task() { .await .unwrap() .unwrap(); - assert_eq!(compact_task.get_task_id(), 3); + assert_eq!(compact_task.task_id, 3); // Finish the task and succeed. assert!(hummock_manager @@ -295,9 +288,9 @@ async fn test_hummock_table() { .iter() .chain(levels.levels.iter()) .flat_map(|level| level.table_infos.iter()) - .map(|info| info.get_object_id()) + .map(|info| info.object_id) .sorted() - .cmp(original_tables.iter().map(|ot| ot.get_object_id()).sorted()) + .cmp(original_tables.iter().map(|ot| ot.object_id).sorted()) ); // Confirm tables got are equal to original tables @@ -764,14 +757,7 @@ async fn test_print_compact_task() { .await .unwrap() .unwrap(); - assert_eq!( - compact_task - .get_input_ssts() - .first() - .unwrap() - .get_level_idx(), - 0 - ); + assert_eq!(compact_task.input_ssts.first().unwrap().level_idx, 0); let s = compact_task_to_string(&compact_task); assert!(s.contains("Compaction task id: 1, group-id: 2, type: Dynamic, target level: 0")); @@ -793,7 +779,7 @@ async fn test_invalid_sst_id() { // reject due to invalid context id let sst_to_worker = ssts .iter() - .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), WorkerId::MAX)) + .map(|LocalSstableInfo { sst_info, .. }| (sst_info.object_id, WorkerId::MAX)) .collect(); let error = hummock_manager .commit_epoch_for_test(epoch, ssts.clone(), sst_to_worker) @@ -803,7 +789,7 @@ async fn test_invalid_sst_id() { let sst_to_worker = ssts .iter() - .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), context_id)) + .map(|LocalSstableInfo { sst_info, .. }| (sst_info.object_id, context_id)) .collect(); hummock_manager .commit_epoch_for_test(epoch, ssts, sst_to_worker) @@ -943,15 +929,8 @@ async fn test_hummock_compaction_task_heartbeat() { .unwrap() .unwrap(); - assert_eq!( - compact_task - .get_input_ssts() - .first() - .unwrap() - .get_level_idx(), - 0 - ); - assert_eq!(compact_task.get_task_id(), 2); + assert_eq!(compact_task.input_ssts.first().unwrap().level_idx, 0); + assert_eq!(compact_task.task_id, 2); for i in 0..10 { // send heartbeats to the task over 2.5 seconds @@ -985,7 +964,7 @@ async fn test_hummock_compaction_task_heartbeat() { .unwrap() .unwrap(); - assert_eq!(compact_task.get_task_id(), 3); + assert_eq!(compact_task.task_id, 3); // Cancel the task after heartbeat has triggered and fail. @@ -1063,15 +1042,8 @@ async fn test_hummock_compaction_task_heartbeat_removal_on_node_removal() { .unwrap() .unwrap(); - assert_eq!( - compact_task - .get_input_ssts() - .first() - .unwrap() - .get_level_idx(), - 0 - ); - assert_eq!(compact_task.get_task_id(), 2); + assert_eq!(compact_task.input_ssts.first().unwrap().level_idx, 0); + assert_eq!(compact_task.task_id, 2); // send heartbeats to the task immediately let req = CompactTaskProgress { @@ -1106,8 +1078,8 @@ async fn test_extend_objects_to_delete() { .iter() .map(|ssts| { ssts.iter() - .max_by_key(|s| s.get_object_id()) - .map(|s| s.get_object_id()) + .max_by_key(|s| s.object_id) + .map(|s| s.object_id) .unwrap() }) .max() @@ -1116,7 +1088,7 @@ async fn test_extend_objects_to_delete() { let all_object_ids = sst_infos .iter() .flatten() - .map(|s| s.get_object_id()) + .map(|s| s.object_id) .chain(max_committed_object_id + 1..=max_committed_object_id + orphan_sst_num) .collect_vec(); assert!(hummock_manager.get_objects_to_delete().is_empty()); @@ -1233,11 +1205,11 @@ async fn test_version_stats() { sst_info: SstableInfo { object_id: sst_ids[idx], sst_id: sst_ids[idx], - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(1, 1, 1), - right: iterator_test_key_of_epoch(1, 1, 1), + key_range: KeyRange { + left: iterator_test_key_of_epoch(1, 1, 1).into(), + right: iterator_test_key_of_epoch(1, 1, 1).into(), right_exclusive: false, - }), + }, file_size: 1024 * 1024 * 1024, table_ids: table_ids.clone(), ..Default::default() @@ -1250,7 +1222,7 @@ async fn test_version_stats() { .collect_vec(); let sst_to_worker = ssts .iter() - .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), worker_node.id)) + .map(|LocalSstableInfo { sst_info, .. }| (sst_info.object_id, worker_node.id)) .collect(); hummock_manager .commit_epoch_for_test(epoch, ssts, sst_to_worker) @@ -1342,7 +1314,7 @@ async fn test_split_compaction_group_on_commit() { sst_info: SstableInfo { object_id: 10, sst_id: 10, - key_range: None, + key_range: KeyRange::default(), table_ids: vec![100, 101], min_epoch: 20, max_epoch: 20, @@ -1430,11 +1402,11 @@ async fn test_split_compaction_group_on_demand_basic() { sst_info: SstableInfo { object_id: 10, sst_id: 10, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(100, 1, 20), - right: iterator_test_key_of_epoch(100, 100, 20), + key_range: KeyRange { + left: iterator_test_key_of_epoch(100, 1, 20).into(), + right: iterator_test_key_of_epoch(100, 100, 20).into(), right_exclusive: false, - }), + }, table_ids: vec![100], min_epoch: 20, max_epoch: 20, @@ -1446,11 +1418,11 @@ async fn test_split_compaction_group_on_demand_basic() { sst_info: SstableInfo { object_id: 11, sst_id: 11, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(100, 101, 20), - right: iterator_test_key_of_epoch(101, 100, 20), + key_range: KeyRange { + left: iterator_test_key_of_epoch(100, 101, 20).into(), + right: iterator_test_key_of_epoch(101, 100, 20).into(), right_exclusive: false, - }), + }, table_ids: vec![100, 101], min_epoch: 20, max_epoch: 20, @@ -1528,7 +1500,7 @@ async fn test_split_compaction_group_on_demand_non_trivial() { sst_info: SstableInfo { object_id: 10, sst_id: 10, - key_range: None, + key_range: KeyRange::default(), table_ids: vec![100, 101], min_epoch: 20, max_epoch: 20, @@ -1613,11 +1585,11 @@ async fn test_split_compaction_group_trivial_expired() { sst_info: SstableInfo { object_id: 10, sst_id: 10, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(100, 1, 20), - right: iterator_test_key_of_epoch(100, 100, 20), + key_range: KeyRange { + left: iterator_test_key_of_epoch(100, 1, 20).into(), + right: iterator_test_key_of_epoch(100, 100, 20).into(), right_exclusive: false, - }), + }, table_ids: vec![100], min_epoch: 20, max_epoch: 20, @@ -1632,11 +1604,11 @@ async fn test_split_compaction_group_trivial_expired() { table_ids: vec![101], min_epoch: 20, max_epoch: 20, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(101, 1, 20), - right: iterator_test_key_of_epoch(101, 100, 20), + key_range: KeyRange { + left: iterator_test_key_of_epoch(101, 1, 20).into(), + right: iterator_test_key_of_epoch(101, 100, 20).into(), right_exclusive: false, - }), + }, ..Default::default() }, table_stats: Default::default(), @@ -1724,7 +1696,7 @@ async fn test_split_compaction_group_trivial_expired() { vec![SstableInfo { object_id: 12, sst_id: 12, - key_range: None, + key_range: KeyRange::default(), table_ids: vec![100], min_epoch: 20, max_epoch: 20, @@ -1779,11 +1751,11 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { sst_info: SstableInfo { object_id: 10, sst_id: 10, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(1, 1, 1), - right: iterator_test_key_of_epoch(1, 1, 1), + key_range: KeyRange { + left: iterator_test_key_of_epoch(1, 1, 1).into(), + right: iterator_test_key_of_epoch(1, 1, 1).into(), right_exclusive: false, - }), + }, table_ids: vec![100, 101], min_epoch: 20, max_epoch: 20, @@ -1810,22 +1782,22 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { object_id: 11, sst_id: 11, table_ids: vec![100, 101], - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(1, 1, 1), - right: iterator_test_key_of_epoch(1, 1, 1), + key_range: KeyRange { + left: iterator_test_key_of_epoch(1, 1, 1).into(), + right: iterator_test_key_of_epoch(1, 1, 1).into(), right_exclusive: false, - }), + }, ..Default::default() }, SstableInfo { object_id: 12, sst_id: 12, table_ids: vec![100], - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(1, 2, 2), - right: iterator_test_key_of_epoch(1, 2, 2), + key_range: KeyRange { + left: iterator_test_key_of_epoch(1, 2, 2).into(), + right: iterator_test_key_of_epoch(1, 2, 2).into(), right_exclusive: false, - }), + }, ..Default::default() }, ], @@ -1864,7 +1836,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { assert_eq!( current_version.get_compaction_group_levels(2).levels[base_level - 1].table_infos[0] .object_id, - sst_1.sst_info.get_object_id() + 1, + sst_1.sst_info.object_id + 1, ); assert_eq!( current_version.get_compaction_group_levels(2).levels[base_level - 1].table_infos[0] @@ -1914,11 +1886,11 @@ async fn test_compaction_task_expiration_due_to_split_group() { sst_info: SstableInfo { object_id: 10, sst_id: 10, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(1, 1, 1), - right: iterator_test_key_of_epoch(1, 1, 1), + key_range: KeyRange { + left: iterator_test_key_of_epoch(1, 1, 1).into(), + right: iterator_test_key_of_epoch(1, 1, 1).into(), right_exclusive: false, - }), + }, table_ids: vec![100, 101], min_epoch: 20, max_epoch: 20, @@ -1930,11 +1902,11 @@ async fn test_compaction_task_expiration_due_to_split_group() { sst_info: SstableInfo { object_id: 11, sst_id: 11, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(1, 1, 1), - right: iterator_test_key_of_epoch(1, 1, 1), + key_range: KeyRange { + left: iterator_test_key_of_epoch(1, 1, 1).into(), + right: iterator_test_key_of_epoch(1, 1, 1).into(), right_exclusive: false, - }), + }, table_ids: vec![101], min_epoch: 20, max_epoch: 20, @@ -2293,11 +2265,11 @@ async fn test_unregister_moved_table() { sst_info: SstableInfo { object_id: 10, sst_id: 10, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(100, 1, 20), - right: iterator_test_key_of_epoch(100, 100, 20), + key_range: KeyRange { + left: iterator_test_key_of_epoch(100, 1, 20).into(), + right: iterator_test_key_of_epoch(100, 100, 20).into(), right_exclusive: false, - }), + }, table_ids: vec![100], min_epoch: 20, max_epoch: 20, @@ -2309,11 +2281,11 @@ async fn test_unregister_moved_table() { sst_info: SstableInfo { object_id: 11, sst_id: 11, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(100, 101, 20), - right: iterator_test_key_of_epoch(101, 100, 20), + key_range: KeyRange { + left: iterator_test_key_of_epoch(100, 101, 20).into(), + right: iterator_test_key_of_epoch(101, 100, 20).into(), right_exclusive: false, - }), + }, table_ids: vec![100, 101], min_epoch: 20, max_epoch: 20, diff --git a/src/meta/src/hummock/manager/time_travel.rs b/src/meta/src/hummock/manager/time_travel.rs index 788ab693579c..6ac687ad39c0 100644 --- a/src/meta/src/hummock/manager/time_travel.rs +++ b/src/meta/src/hummock/manager/time_travel.rs @@ -18,6 +18,7 @@ use anyhow::anyhow; use itertools::Itertools; use risingwave_common::util::epoch::Epoch; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::time_travel::{ refill_version, IncompleteHummockVersion, IncompleteHummockVersionDelta, }; @@ -25,11 +26,12 @@ use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{ CompactionGroupId, HummockEpoch, HummockSstableId, HummockSstableObjectId, }; +use risingwave_meta_model_v2::hummock_sstable_info::SstableInfoV2Backend; use risingwave_meta_model_v2::{ hummock_epoch_to_version, hummock_sstable_info, hummock_time_travel_delta, hummock_time_travel_version, }; -use risingwave_pb::hummock::{PbHummockVersion, PbHummockVersionDelta, PbSstableInfo}; +use risingwave_pb::hummock::{PbHummockVersion, PbHummockVersionDelta}; use sea_orm::sea_query::OnConflict; use sea_orm::ActiveValue::Set; use sea_orm::{ @@ -323,7 +325,7 @@ impl HummockManager { .all(&sql_store.conn) .await?; for sst_info in sst_infos { - let sst_info = sst_info.sstable_info.to_protobuf(); + let sst_info: SstableInfo = sst_info.sstable_info.to_protobuf().into(); sst_id_to_info.insert(sst_info.sst_id, sst_info); } } @@ -346,7 +348,7 @@ impl HummockManager { skip_sst_ids: &HashSet, ) -> Result>> { async fn write_sstable_infos( - sst_infos: impl Iterator, + sst_infos: impl Iterator, txn: &DatabaseTransaction, ) -> Result { let mut count = 0; @@ -354,7 +356,7 @@ impl HummockManager { let m = hummock_sstable_info::ActiveModel { sst_id: Set(sst_info.sst_id.try_into().unwrap()), object_id: Set(sst_info.object_id.try_into().unwrap()), - sstable_info: Set(sst_info.into()), + sstable_info: Set(SstableInfoV2Backend::from(&sst_info.to_protobuf())), }; hummock_sstable_info::Entity::insert(m) .on_conflict( diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 878870a6d8d2..b6a41e8ffa8a 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -16,14 +16,14 @@ use std::collections::{BTreeMap, HashMap}; use std::ops::{Deref, DerefMut}; use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::change_log::ChangeLogDelta; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_watermark::TableWatermarks; -use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; -use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, HummockVersionId}; -use risingwave_pb::hummock::group_delta::DeltaType; -use risingwave_pb::hummock::hummock_version_delta::ChangeLogDelta; -use risingwave_pb::hummock::{ - GroupDelta, HummockVersionStats, IntraLevelDelta, SstableInfo, StateTableInfoDelta, +use risingwave_hummock_sdk::version::{ + GroupDelta, HummockVersion, HummockVersionDelta, IntraLevelDelta, }; +use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, HummockVersionId}; +use risingwave_pb::hummock::{HummockVersionStats, StateTableInfoDelta}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use crate::manager::NotificationManager; @@ -121,7 +121,6 @@ impl<'a> HummockVersionTransaction<'a> { new_version_delta.max_committed_epoch = epoch; new_version_delta.new_table_watermarks = new_table_watermarks; new_version_delta.change_log_delta = change_log_delta; - // Append SSTs to a new version. for (compaction_group_id, inserted_table_infos) in commit_sstables { let group_deltas = &mut new_version_delta @@ -130,14 +129,14 @@ impl<'a> HummockVersionTransaction<'a> { .or_default() .group_deltas; let l0_sub_level_id = epoch; - let group_delta = GroupDelta { - delta_type: Some(DeltaType::IntraLevel(IntraLevelDelta { - level_idx: 0, - inserted_table_infos, - l0_sub_level_id, - ..Default::default() - })), - }; + let group_delta = GroupDelta::IntraLevel(IntraLevelDelta::new( + 0, + l0_sub_level_id, + vec![], // default + inserted_table_infos, + 0, // default + )); + group_deltas.push(group_delta); } diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index 0603325c2e62..4777a203bb3c 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -21,6 +21,7 @@ use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ get_compaction_group_ids, get_table_compaction_group_id_mapping, BranchedSstInfo, }; use risingwave_hummock_sdk::compaction_group::StateTableId; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::add_prost_table_stats_map; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{ @@ -30,8 +31,7 @@ use risingwave_hummock_sdk::{ use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ - HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, HummockVersionStats, SstableInfo, - TableStats, + HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, HummockVersionStats, TableStats, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -381,7 +381,7 @@ fn estimate_table_stats(sst: &SstableInfo) -> HashMap { let mut changes: HashMap = HashMap::default(); let weighted_value = |value: i64| -> i64 { (value as f64 / sst.table_ids.len() as f64).ceil() as i64 }; - let key_range = sst.key_range.as_ref().unwrap(); + let key_range = &sst.key_range; let estimated_key_size: u64 = (key_range.left.len() + key_range.right.len()) as u64 / 2; let mut estimated_total_key_size = estimated_key_size * sst.total_key_count; if estimated_total_key_size > sst.uncompressed_file_size { @@ -403,13 +403,13 @@ mod tests { use std::collections::HashMap; use std::sync::Arc; + use risingwave_hummock_sdk::key_range::KeyRange; + use risingwave_hummock_sdk::level::{Level, Levels, OverlappingLevel}; + use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{CompactionGroupId, HummockVersionId}; - use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::write_limits::WriteLimit; - use risingwave_pb::hummock::{ - HummockPinnedVersion, HummockVersionStats, KeyRange, Level, OverlappingLevel, SstableInfo, - }; + use risingwave_pb::hummock::{HummockPinnedVersion, HummockVersionStats}; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::manager::context::ContextInfo; @@ -540,11 +540,11 @@ mod tests { #[test] fn test_estimate_table_stats() { let sst = SstableInfo { - key_range: Some(KeyRange { - left: vec![1; 10], - right: vec![1; 20], + key_range: KeyRange { + left: vec![1; 10].into(), + right: vec![1; 20].into(), ..Default::default() - }), + }, table_ids: vec![1, 2, 3], total_key_count: 6000, uncompressed_file_size: 6_000_000, @@ -602,11 +602,11 @@ mod tests { #[test] fn test_estimate_table_stats_large_key_range() { let sst = SstableInfo { - key_range: Some(KeyRange { - left: vec![1; 1000], - right: vec![1; 2000], + key_range: KeyRange { + left: vec![1; 1000].into(), + right: vec![1; 2000].into(), ..Default::default() - }), + }, table_ids: vec![1, 2, 3], total_key_count: 6000, uncompressed_file_size: 60_000, diff --git a/src/meta/src/hummock/metrics_utils.rs b/src/meta/src/hummock/metrics_utils.rs index 3779ff5b2be9..992d5b114690 100644 --- a/src/meta/src/hummock/metrics_utils.rs +++ b/src/meta/src/hummock/metrics_utils.rs @@ -21,10 +21,10 @@ use itertools::{enumerate, Itertools}; use prometheus::core::{AtomicU64, GenericCounter}; use prometheus::IntGauge; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::object_size_map; +use risingwave_hummock_sdk::level::Levels; use risingwave_hummock_sdk::table_stats::PbTableStatsMap; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{CompactionGroupId, HummockContextId, HummockEpoch, HummockVersionId}; -use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ CompactionConfig, HummockPinnedSnapshot, HummockPinnedVersion, HummockVersionStats, LevelType, @@ -228,7 +228,7 @@ pub fn trigger_sst_stat( level.l0.as_ref().map(|l0| { l0.sub_levels .iter() - .filter(|sub_level| sub_level.level_type() == LevelType::Overlapping) + .filter(|sub_level| sub_level.level_type == LevelType::Overlapping) .count() }) }) @@ -241,7 +241,7 @@ pub fn trigger_sst_stat( level.l0.as_ref().map(|l0| { l0.sub_levels .iter() - .filter(|sub_level| sub_level.level_type() == LevelType::Nonoverlapping) + .filter(|sub_level| sub_level.level_type == LevelType::Nonoverlapping) .count() }) }) @@ -255,7 +255,7 @@ pub fn trigger_sst_stat( l0.sub_levels .iter() .filter(|sub_level| { - sub_level.level_type() == LevelType::Nonoverlapping + sub_level.level_type == LevelType::Nonoverlapping && sub_level.vnode_partition_count > 0 }) .count() @@ -481,16 +481,16 @@ pub fn trigger_lsm_stat( { // compact_level_compression_ratio let level_compression_ratio = levels - .get_levels() + .levels .iter() .map(|level| { - let ratio = if level.get_uncompressed_file_size() == 0 { + let ratio = if level.uncompressed_file_size == 0 { 0.0 } else { - level.get_total_file_size() as f64 / level.get_uncompressed_file_size() as f64 + level.total_file_size as f64 / level.uncompressed_file_size as f64 }; - (level.get_level_idx(), ratio) + (level.level_idx, ratio) }) .collect_vec(); diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 95e5ad9a368c..8499605e04fb 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -22,8 +22,11 @@ use async_trait::async_trait; use fail::fail_point; use futures::stream::BoxStream; use futures::{Stream, StreamExt}; +use itertools::Itertools; use risingwave_hummock_sdk::change_log::build_table_change_log_delta; +use risingwave_hummock_sdk::compact_task::CompactTask; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, @@ -34,7 +37,7 @@ use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::subscribe_compaction_event_request::{Event, ReportTask}; use risingwave_pb::hummock::subscribe_compaction_event_response::Event as ResponseEvent; use risingwave_pb::hummock::{ - compact_task, CompactTask, HummockSnapshot, PbHummockVersion, SubscribeCompactionEventRequest, + compact_task, HummockSnapshot, PbHummockVersion, SubscribeCompactionEventRequest, SubscribeCompactionEventResponse, VacuumTask, }; use risingwave_rpc_client::error::{Result, RpcError}; @@ -162,7 +165,7 @@ impl HummockMetaClient for MockHummockMetaClient { let sst_to_worker = sync_result .uncommitted_ssts .iter() - .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), self.context_id)) + .map(|LocalSstableInfo { sst_info, .. }| (sst_info.object_id, self.context_id)) .collect(); let new_table_watermark = sync_result.table_watermarks; let table_change_log = build_table_change_log_delta( @@ -293,7 +296,7 @@ impl HummockMetaClient for MockHummockMetaClient { .unwrap() { let resp = SubscribeCompactionEventResponse { - event: Some(ResponseEvent::CompactTask(task)), + event: Some(ResponseEvent::CompactTask(task.into())), create_at: SystemTime::now() .duration_since(std::time::UNIX_EPOCH) .expect("Clock may have gone backwards") @@ -323,7 +326,10 @@ impl HummockMetaClient for MockHummockMetaClient { .report_compact_task( task_id, TaskStatus::try_from(task_status).unwrap(), - sorted_output_ssts, + sorted_output_ssts + .into_iter() + .map(SstableInfo::from) + .collect_vec(), Some(table_stats_change), ) .await diff --git a/src/meta/src/hummock/model/ext/hummock.rs b/src/meta/src/hummock/model/ext/hummock.rs index 6f40523f1e72..c82df1013737 100644 --- a/src/meta/src/hummock/model/ext/hummock.rs +++ b/src/meta/src/hummock/model/ext/hummock.rs @@ -226,7 +226,7 @@ impl Transactional for HummockVersionDelta { max_committed_epoch: Set(self.max_committed_epoch.try_into().unwrap()), safe_epoch: Set(self.visible_table_safe_epoch().try_into().unwrap()), trivial_move: Set(self.trivial_move), - full_version_delta: Set(FullVersionDelta::from(&self.to_protobuf())), + full_version_delta: Set(FullVersionDelta::from(&self.into())), }; hummock_version_delta::Entity::insert(m) .on_conflict( diff --git a/src/meta/src/hummock/model/version_delta.rs b/src/meta/src/hummock/model/version_delta.rs index 1a87b9d45698..ed2be4761aca 100644 --- a/src/meta/src/hummock/model/version_delta.rs +++ b/src/meta/src/hummock/model/version_delta.rs @@ -29,7 +29,7 @@ impl MetadataModel for HummockVersionDelta { } fn to_protobuf(&self) -> Self::PbType { - self.to_protobuf() + self.into() } fn from_protobuf(prost: Self::PbType) -> Self { diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 891a1d8c6ab7..84c19cfd4d93 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -18,11 +18,15 @@ use std::collections::{BTreeSet, HashMap}; use std::sync::Arc; use std::time::Duration; +use bytes::Bytes; use itertools::Itertools; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::key_with_epoch; +use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::level::Levels; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionStateTableInfo}; use risingwave_hummock_sdk::{ @@ -30,8 +34,7 @@ use risingwave_hummock_sdk::{ }; use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::hummock::compact_task::TaskStatus; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{CompactionConfig, KeyRange, SstableInfo}; +use risingwave_pb::hummock::CompactionConfig; use risingwave_pb::meta::add_worker_node_request::Property; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; @@ -71,7 +74,7 @@ pub async fn add_test_tables( let ssts = to_local_sstable_info(&test_tables); let sst_to_worker = ssts .iter() - .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), context_id)) + .map(|LocalSstableInfo { sst_info, .. }| (sst_info.object_id, context_id)) .collect(); hummock_manager .commit_epoch_for_test(epoch, ssts, sst_to_worker) @@ -146,7 +149,7 @@ pub async fn add_test_tables( let ssts = to_local_sstable_info(&test_tables_3); let sst_to_worker = ssts .iter() - .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), context_id)) + .map(|LocalSstableInfo { sst_info, .. }| (sst_info.object_id, context_id)) .collect(); hummock_manager .commit_epoch_for_test(epoch, ssts, sst_to_worker) @@ -165,21 +168,21 @@ pub fn generate_test_sstables_with_table_id( sst_info.push(SstableInfo { object_id: sst_id, sst_id, - key_range: Some(KeyRange { - left: key_with_epoch( + key_range: KeyRange { + left: Bytes::from(key_with_epoch( format!("{:03}\0\0_key_test_{:05}", table_id, i + 1) .as_bytes() .to_vec(), epoch, - ), - right: key_with_epoch( + )), + right: Bytes::from(key_with_epoch( format!("{:03}\0\0_key_test_{:05}", table_id, (i + 1) * 10) .as_bytes() .to_vec(), epoch, - ), + )), right_exclusive: false, - }), + }, file_size: 2, table_ids: vec![table_id], uncompressed_file_size: 2, @@ -196,11 +199,11 @@ pub fn generate_test_tables(epoch: u64, sst_ids: Vec) -> sst_info.push(SstableInfo { object_id: sst_id, sst_id, - key_range: Some(KeyRange { - left: iterator_test_key_of_epoch(sst_id, i + 1, epoch), - right: iterator_test_key_of_epoch(sst_id, (i + 1) * 10, epoch), + key_range: KeyRange { + left: Bytes::from(iterator_test_key_of_epoch(sst_id, i + 1, epoch)), + right: Bytes::from(iterator_test_key_of_epoch(sst_id, (i + 1) * 10, epoch)), right_exclusive: false, - }), + }, file_size: 2, table_ids: vec![sst_id as u32, sst_id as u32 * 10000], uncompressed_file_size: 2, @@ -275,7 +278,7 @@ pub fn iterator_test_key_of_epoch( pub fn get_sorted_object_ids(sstables: &[SstableInfo]) -> Vec { sstables .iter() - .map(|table| table.get_object_id()) + .map(|table| table.object_id) .sorted() .collect_vec() } @@ -294,7 +297,7 @@ pub fn get_sorted_committed_object_ids( .levels .iter() .chain(levels.l0.as_ref().unwrap().sub_levels.iter()) - .flat_map(|levels| levels.table_infos.iter().map(|info| info.get_object_id())) + .flat_map(|levels| levels.table_infos.iter().map(|info| info.object_id)) .sorted() .collect_vec() } @@ -383,7 +386,7 @@ pub async fn commit_from_meta_node( ) -> crate::hummock::error::Result<()> { let sst_to_worker = ssts .iter() - .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), META_NODE_ID)) + .map(|LocalSstableInfo { sst_info, .. }| (sst_info.object_id, META_NODE_ID)) .collect(); hummock_manager_ref .commit_epoch_for_test(epoch, ssts, sst_to_worker) @@ -400,7 +403,7 @@ pub async fn add_ssts( let ssts = to_local_sstable_info(&test_tables); let sst_to_worker = ssts .iter() - .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), context_id)) + .map(|LocalSstableInfo { sst_info, .. }| (sst_info.object_id, context_id)) .collect(); hummock_manager .commit_epoch_for_test(epoch, ssts, sst_to_worker) diff --git a/src/meta/src/hummock/vacuum.rs b/src/meta/src/hummock/vacuum.rs index 5dffb35dfc6c..08f227dd3184 100644 --- a/src/meta/src/hummock/vacuum.rs +++ b/src/meta/src/hummock/vacuum.rs @@ -272,7 +272,7 @@ mod tests { .first() .unwrap() .iter() - .map(|s| s.get_object_id()) + .map(|s| s.object_id) .collect_vec(), }) .await diff --git a/src/meta/src/manager/diagnose.rs b/src/meta/src/manager/diagnose.rs index 6ec3ab263f54..1d6fcdd7f8fe 100644 --- a/src/meta/src/manager/diagnose.rs +++ b/src/meta/src/manager/diagnose.rs @@ -21,8 +21,8 @@ use itertools::Itertools; use prometheus_http_query::response::Data::Vector; use risingwave_common::types::Timestamptz; use risingwave_common::util::StackTraceResponseExt; +use risingwave_hummock_sdk::level::Level; use risingwave_pb::common::WorkerType; -use risingwave_pb::hummock::Level; use risingwave_pb::meta::event_log::Event; use risingwave_pb::meta::EventLog; use risingwave_pb::monitor_service::StackTraceResponse; diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 87410d0b9bfb..25cf2222b440 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -1034,7 +1034,7 @@ impl MetaClient { version_delta: HummockVersionDelta, ) -> Result<(HummockVersion, Vec)> { let req = ReplayVersionDeltaRequest { - version_delta: Some(version_delta.to_protobuf()), + version_delta: Some(version_delta.into()), }; let resp = self.inner.replay_version_delta(req).await?; Ok(( diff --git a/src/storage/backup/src/meta_snapshot_v1.rs b/src/storage/backup/src/meta_snapshot_v1.rs index f94e3e1446eb..b66609ed4eb2 100644 --- a/src/storage/backup/src/meta_snapshot_v1.rs +++ b/src/storage/backup/src/meta_snapshot_v1.rs @@ -22,7 +22,7 @@ use risingwave_hummock_sdk::version::HummockVersion; use risingwave_pb::catalog::{ Connection, Database, Function, Index, Schema, Sink, Source, Subscription, Table, View, }; -use risingwave_pb::hummock::{CompactionGroup, HummockVersionStats}; +use risingwave_pb::hummock::{CompactionGroup, HummockVersionStats, PbHummockVersion}; use risingwave_pb::meta::{SystemParams, TableFragments}; use risingwave_pb::user::UserInfo; @@ -129,7 +129,7 @@ impl ClusterMetadata { let default_cf_values = self.default_cf.values().collect_vec(); Self::encode_prost_message_list(&default_cf_keys, buf); Self::encode_prost_message_list(&default_cf_values, buf); - Self::encode_prost_message(&self.hummock_version.to_protobuf(), buf); + Self::encode_prost_message(&PbHummockVersion::from(&self.hummock_version), buf); Self::encode_prost_message(&self.version_stats, buf); Self::encode_prost_message_list(&self.compaction_groups.iter().collect_vec(), buf); Self::encode_prost_message_list(&self.table_fragments.iter().collect_vec(), buf); diff --git a/src/storage/benches/bench_compactor.rs b/src/storage/benches/bench_compactor.rs index 08da1438709c..e5f929a312a8 100644 --- a/src/storage/benches/bench_compactor.rs +++ b/src/storage/benches/bench_compactor.rs @@ -27,9 +27,11 @@ use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAw use risingwave_common::util::value_encoding::ValueRowSerializer; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::{InMemObjectStore, ObjectStore, ObjectStoreImpl}; -use risingwave_pb::hummock::{compact_task, SstableInfo, TableSchema}; +use risingwave_pb::hummock::compact_task::PbTaskType; +use risingwave_pb::hummock::PbTableSchema; use risingwave_storage::hummock::compactor::compactor_runner::compact_and_build_sst; use risingwave_storage::hummock::compactor::{ ConcatSstableIterator, DummyCompactionFilter, TaskConfig, TaskProgress, @@ -280,7 +282,7 @@ async fn compact>( gc_delete_keys: false, watermark: 0, stats_target_table_ids: None, - task_type: compact_task::TaskType::Dynamic, + task_type: PbTaskType::Dynamic, use_block_based_filter: true, ..Default::default() }); @@ -432,7 +434,7 @@ fn bench_drop_column_compaction_impl(c: &mut Criterion, column_num: usize) { gc_delete_keys: false, watermark: 0, stats_target_table_ids: None, - task_type: compact_task::TaskType::Dynamic, + task_type: PbTaskType::Dynamic, use_block_based_filter: true, table_schemas: vec![].into_iter().collect(), ..Default::default() @@ -441,13 +443,13 @@ fn bench_drop_column_compaction_impl(c: &mut Criterion, column_num: usize) { let mut task_config_schema = task_config_no_schema.clone(); task_config_schema.table_schemas.insert( 10, - TableSchema { + PbTableSchema { column_ids: (0..column_num as i32).collect(), }, ); task_config_schema.table_schemas.insert( 11, - TableSchema { + PbTableSchema { column_ids: (0..column_num as i32).collect(), }, ); @@ -455,13 +457,13 @@ fn bench_drop_column_compaction_impl(c: &mut Criterion, column_num: usize) { let mut task_config_schema_cause_drop = task_config_no_schema.clone(); task_config_schema_cause_drop.table_schemas.insert( 10, - TableSchema { + PbTableSchema { column_ids: (0..column_num as i32 / 2).collect(), }, ); task_config_schema_cause_drop.table_schemas.insert( 11, - TableSchema { + PbTableSchema { column_ids: (0..column_num as i32 / 2).collect(), }, ); diff --git a/src/storage/benches/bench_multi_builder.rs b/src/storage/benches/bench_multi_builder.rs index 08d744189a1d..730534a81831 100644 --- a/src/storage/benches/bench_multi_builder.rs +++ b/src/storage/benches/bench_multi_builder.rs @@ -25,10 +25,10 @@ use rand::random; use risingwave_common::catalog::TableId; use risingwave_common::config::{MetricLevel, ObjectStoreConfig}; use risingwave_hummock_sdk::key::{FullKey, UserKey}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_object_store::object::{ InMemObjectStore, ObjectStore, ObjectStoreImpl, S3ObjectStore, }; -use risingwave_pb::hummock::SstableInfo; use risingwave_storage::hummock::iterator::{ConcatIterator, ConcatIteratorInner, HummockIterator}; use risingwave_storage::hummock::multi_builder::{CapacitySplitTableBuilder, TableBuilderFactory}; use risingwave_storage::hummock::value::HummockValue; diff --git a/src/storage/hummock_sdk/src/change_log.rs b/src/storage/hummock_sdk/src/change_log.rs index 49224c92091f..885880933ce1 100644 --- a/src/storage/hummock_sdk/src/change_log.rs +++ b/src/storage/hummock_sdk/src/change_log.rs @@ -15,13 +15,70 @@ use std::collections::HashMap; use risingwave_common::catalog::TableId; -use risingwave_pb::hummock::hummock_version_delta::ChangeLogDelta; -use risingwave_pb::hummock::{EpochNewChangeLog, SstableInfo, TableChangeLog as PbTableChangeLog}; +use risingwave_pb::hummock::hummock_version_delta::PbChangeLogDelta; +use risingwave_pb::hummock::{PbEpochNewChangeLog, PbTableChangeLog}; use tracing::warn; +use crate::sstable_info::SstableInfo; + #[derive(Debug, Clone, PartialEq)] pub struct TableChangeLog(pub Vec); +#[derive(Debug, Clone, PartialEq)] +pub struct EpochNewChangeLog { + pub new_value: Vec, + pub old_value: Vec, + pub epochs: Vec, +} + +impl From<&EpochNewChangeLog> for PbEpochNewChangeLog { + fn from(val: &EpochNewChangeLog) -> Self { + Self { + new_value: val.new_value.iter().map(|a| a.clone().into()).collect(), + old_value: val.old_value.iter().map(|a| a.clone().into()).collect(), + epochs: val.epochs.clone(), + } + } +} + +impl From<&PbEpochNewChangeLog> for EpochNewChangeLog { + fn from(value: &PbEpochNewChangeLog) -> Self { + Self { + new_value: value.new_value.iter().map(|a| a.into()).collect(), + old_value: value.old_value.iter().map(|a| a.into()).collect(), + epochs: value.epochs.clone(), + } + } +} + +impl From for PbEpochNewChangeLog { + fn from(val: EpochNewChangeLog) -> Self { + Self { + new_value: val + .new_value + .into_iter() + .map(|a| a.clone().into()) + .collect(), + old_value: val + .old_value + .into_iter() + .map(|a| a.clone().into()) + .collect(), + epochs: val.epochs.clone(), + } + } +} + +impl From for EpochNewChangeLog { + fn from(value: PbEpochNewChangeLog) -> Self { + Self { + new_value: value.new_value.into_iter().map(|a| a.into()).collect(), + old_value: value.old_value.into_iter().map(|a| a.into()).collect(), + epochs: value.epochs.clone(), + } + } +} + impl TableChangeLog { pub fn filter_epoch(&self, (min_epoch, max_epoch): (u64, u64)) -> &[EpochNewChangeLog] { let start = self.0.partition_point(|epoch_change_log| { @@ -56,12 +113,12 @@ impl TableChangeLog { impl TableChangeLog { pub fn to_protobuf(&self) -> PbTableChangeLog { PbTableChangeLog { - change_logs: self.0.clone(), + change_logs: self.0.iter().map(|a| a.into()).collect(), } } pub fn from_protobuf(val: &PbTableChangeLog) -> Self { - Self(val.change_logs.clone()) + Self(val.change_logs.clone().iter().map(|a| a.into()).collect()) } } @@ -108,6 +165,48 @@ pub fn build_table_change_log_delta<'a>( table_change_log } +#[derive(Debug, PartialEq, Clone)] +pub struct ChangeLogDelta { + pub truncate_epoch: u64, + pub new_log: Option, +} + +impl From<&ChangeLogDelta> for PbChangeLogDelta { + fn from(val: &ChangeLogDelta) -> Self { + Self { + truncate_epoch: val.truncate_epoch, + new_log: val.new_log.as_ref().map(|a| a.into()), + } + } +} + +impl From<&PbChangeLogDelta> for ChangeLogDelta { + fn from(val: &PbChangeLogDelta) -> Self { + Self { + truncate_epoch: val.truncate_epoch, + new_log: val.new_log.as_ref().map(|a| a.into()), + } + } +} + +impl From for PbChangeLogDelta { + fn from(val: ChangeLogDelta) -> Self { + Self { + truncate_epoch: val.truncate_epoch, + new_log: val.new_log.map(|a| a.into()), + } + } +} + +impl From for ChangeLogDelta { + fn from(val: PbChangeLogDelta) -> Self { + Self { + truncate_epoch: val.truncate_epoch, + new_log: val.new_log.map(|a| a.into()), + } + } +} + #[cfg(test)] mod tests { use itertools::Itertools; diff --git a/src/storage/hummock_sdk/src/compact.rs b/src/storage/hummock_sdk/src/compact.rs index d6824ab646c4..ff1241447791 100644 --- a/src/storage/hummock_sdk/src/compact.rs +++ b/src/storage/hummock_sdk/src/compact.rs @@ -14,7 +14,10 @@ use std::collections::HashSet; -use risingwave_pb::hummock::{CompactTask, LevelType, SstableInfo}; +use risingwave_pb::hummock::LevelType; + +use crate::compact_task::CompactTask; +use crate::sstable_info::SstableInfo; pub fn compact_task_output_to_string(compact_task: &CompactTask) -> String { use std::fmt::Write; @@ -25,13 +28,13 @@ pub fn compact_task_output_to_string(compact_task: &CompactTask) -> String { "Compaction task id: {:?}, group-id: {:?}, type: {:?}, target level: {:?}, target sub level: {:?} watermark: {:?}, target_file_size: {:?}, splits: {:?}, status: {:?}", compact_task.task_id, compact_task.compaction_group_id, - compact_task.task_type(), + compact_task.task_type, compact_task.target_level, compact_task.target_sub_level_id, compact_task.watermark, compact_task.target_file_size, compact_task.splits.len(), - compact_task.task_status() + compact_task.task_status ) .unwrap(); s.push_str("Output: \n"); @@ -50,7 +53,7 @@ pub fn compact_task_to_string(compact_task: &CompactTask) -> String { "Compaction task id: {:?}, group-id: {:?}, type: {:?}, target level: {:?}, target sub level: {:?} watermark: {:?}, target_file_size: {:?}, splits: {:?}", compact_task.task_id, compact_task.compaction_group_id, - compact_task.task_type(), + compact_task.task_type, compact_task.target_level, compact_task.target_sub_level_id, compact_task.watermark, @@ -81,7 +84,7 @@ pub fn compact_task_to_string(compact_task: &CompactTask) -> String { if table.total_key_count != 0 { format!( "[id: {}, obj_id: {} {}KB stale_ratio {}]", - table.get_sst_id(), + table.sst_id, table.object_id, table.file_size / 1024, (table.stale_key_count * 100 / table.total_key_count), @@ -89,7 +92,7 @@ pub fn compact_task_to_string(compact_task: &CompactTask) -> String { } else { format!( "[id: {}, obj_id: {} {}KB]", - table.get_sst_id(), + table.sst_id, table.object_id, table.file_size / 1024, ) @@ -118,16 +121,16 @@ pub fn compact_task_to_string(compact_task: &CompactTask) -> String { pub fn append_sstable_info_to_string(s: &mut String, sstable_info: &SstableInfo) { use std::fmt::Write; - let key_range = sstable_info.key_range.as_ref().unwrap(); + let key_range = &sstable_info.key_range; let left_str = if key_range.left.is_empty() { "-inf".to_string() } else { - hex::encode(key_range.left.as_slice()) + hex::encode(&key_range.left) }; let right_str = if key_range.right.is_empty() { "+inf".to_string() } else { - hex::encode(key_range.right.as_slice()) + hex::encode(&key_range.right) }; let stale_ratio = (sstable_info.stale_key_count * 100) @@ -136,8 +139,8 @@ pub fn append_sstable_info_to_string(s: &mut String, sstable_info: &SstableInfo) writeln!( s, "SstableInfo: object id={}, SST id={}, KeyRange=[{:?},{:?}], table_ids: {:?}, size={}KB, stale_ratio={}%, bloom_filter_kind {:?}", - sstable_info.get_object_id(), - sstable_info.get_sst_id(), + sstable_info.object_id, + sstable_info.sst_id, left_str, right_str, sstable_info.table_ids, @@ -158,8 +161,8 @@ pub fn statistics_compact_task(task: &CompactTask) -> CompactTaskStatistics { total_file_count += level.table_infos.len() as u64; level.table_infos.iter().for_each(|sst| { - total_file_size += sst.get_file_size(); - total_uncompressed_file_size += sst.get_uncompressed_file_size(); + total_file_size += sst.file_size; + total_uncompressed_file_size += sst.uncompressed_file_size; total_key_count += sst.total_key_count; }); } @@ -200,7 +203,7 @@ pub fn estimate_memory_for_compact_task( // input for level in &task.input_ssts { - if level.level_type() == LevelType::Nonoverlapping { + if level.level_type == LevelType::Nonoverlapping { let mut cur_level_max_sst_meta_size = 0; for sst in &level.table_infos { let meta_size = sst.file_size - sst.meta_offset; diff --git a/src/storage/hummock_sdk/src/compact_task.rs b/src/storage/hummock_sdk/src/compact_task.rs new file mode 100644 index 000000000000..61f96e1ee3b5 --- /dev/null +++ b/src/storage/hummock_sdk/src/compact_task.rs @@ -0,0 +1,412 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{BTreeMap, HashMap}; +use std::mem::size_of; + +use itertools::Itertools; +use risingwave_pb::hummock::compact_task::{PbTaskStatus, PbTaskType, TaskStatus}; +use risingwave_pb::hummock::subscribe_compaction_event_request::PbReportTask; +use risingwave_pb::hummock::{ + PbCompactTask, PbKeyRange, PbTableOption, PbTableSchema, PbTableStats, PbValidationTask, +}; + +use crate::key_range::KeyRange; +use crate::level::InputLevel; +use crate::sstable_info::SstableInfo; +use crate::table_watermark::TableWatermarks; + +#[derive(Clone, PartialEq, Default, Debug)] +pub struct CompactTask { + /// SSTs to be compacted, which will be removed from LSM after compaction + pub input_ssts: Vec, + /// In ideal case, the compaction will generate `splits.len()` tables which have key range + /// corresponding to that in `splits`, respectively + pub splits: Vec, + /// low watermark in 'ts-aware compaction' + pub watermark: u64, + /// compaction output, which will be added to `target_level` of LSM after compaction + pub sorted_output_ssts: Vec, + /// task id assigned by hummock storage service + pub task_id: u64, + /// compaction output will be added to `target_level` of LSM after compaction + pub target_level: u32, + pub gc_delete_keys: bool, + /// Lbase in LSM + pub base_level: u32, + pub task_status: PbTaskStatus, + /// compaction group the task belongs to + pub compaction_group_id: u64, + /// `existing_table_ids` for compaction drop key + pub existing_table_ids: Vec, + pub compression_algorithm: u32, + pub target_file_size: u64, + pub compaction_filter_mask: u32, + pub table_options: BTreeMap, + pub current_epoch_time: u64, + pub target_sub_level_id: u64, + /// Identifies whether the task is `space_reclaim`, if the `compact_task_type` increases, it will be refactored to enum + pub task_type: PbTaskType, + /// Deprecated. use `table_vnode_partition` instead; + pub split_by_state_table: bool, + /// Compaction needs to cut the state table every time 1/weight of vnodes in the table have been processed. + /// Deprecated. use `table_vnode_partition` instead; + pub split_weight_by_vnode: u32, + pub table_vnode_partition: BTreeMap, + /// The table watermark of any table id. In compaction we only use the table watermarks on safe epoch, + /// so we only need to include the table watermarks on safe epoch to reduce the size of metadata. + pub table_watermarks: BTreeMap, + + pub table_schemas: BTreeMap, + + pub max_sub_compaction: u32, +} + +impl CompactTask { + pub fn estimated_encode_len(&self) -> usize { + self.input_ssts + .iter() + .map(|input_level| input_level.estimated_encode_len()) + .sum::() + + self + .splits + .iter() + .map(|split| split.left.len() + split.right.len() + size_of::()) + .sum::() + + size_of::() + + self + .sorted_output_ssts + .iter() + .map(|sst| sst.estimated_encode_len()) + .sum::() + + size_of::() + + size_of::() + + size_of::() + + size_of::() + + size_of::() + + size_of::() + + self.existing_table_ids.len() * size_of::() + + size_of::() + + size_of::() + + size_of::() + + self.table_options.len() * size_of::() + + size_of::() + + size_of::() + + size_of::() + + size_of::() + + size_of::() + + self.table_vnode_partition.len() * size_of::() + + self + .table_watermarks + .values() + .map(|table_watermark| size_of::() + table_watermark.estimated_encode_len()) + .sum::() + } +} + +impl From for CompactTask { + #[expect(deprecated)] + fn from(pb_compact_task: PbCompactTask) -> Self { + Self { + input_ssts: pb_compact_task + .input_ssts + .into_iter() + .map(InputLevel::from) + .collect_vec(), + splits: pb_compact_task + .splits + .into_iter() + .map(|pb_keyrange| KeyRange { + left: pb_keyrange.left.into(), + right: pb_keyrange.right.into(), + right_exclusive: pb_keyrange.right_exclusive, + }) + .collect_vec(), + watermark: pb_compact_task.watermark, + sorted_output_ssts: pb_compact_task + .sorted_output_ssts + .into_iter() + .map(SstableInfo::from) + .collect_vec(), + task_id: pb_compact_task.task_id, + target_level: pb_compact_task.target_level, + gc_delete_keys: pb_compact_task.gc_delete_keys, + base_level: pb_compact_task.base_level, + task_status: TaskStatus::try_from(pb_compact_task.task_status).unwrap(), + compaction_group_id: pb_compact_task.compaction_group_id, + existing_table_ids: pb_compact_task.existing_table_ids.clone(), + compression_algorithm: pb_compact_task.compression_algorithm, + target_file_size: pb_compact_task.target_file_size, + compaction_filter_mask: pb_compact_task.compaction_filter_mask, + table_options: pb_compact_task.table_options.clone(), + current_epoch_time: pb_compact_task.current_epoch_time, + target_sub_level_id: pb_compact_task.target_sub_level_id, + task_type: PbTaskType::try_from(pb_compact_task.task_type).unwrap(), + split_by_state_table: pb_compact_task.split_by_state_table, + split_weight_by_vnode: pb_compact_task.split_weight_by_vnode, + table_vnode_partition: pb_compact_task.table_vnode_partition.clone(), + table_watermarks: pb_compact_task + .table_watermarks + .into_iter() + .map(|(table_id, pb_table_watermark)| { + (table_id, TableWatermarks::from(pb_table_watermark)) + }) + .collect(), + table_schemas: pb_compact_task.table_schemas, + max_sub_compaction: pb_compact_task.max_sub_compaction, + } + } +} + +impl From<&PbCompactTask> for CompactTask { + #[expect(deprecated)] + fn from(pb_compact_task: &PbCompactTask) -> Self { + Self { + input_ssts: pb_compact_task + .input_ssts + .iter() + .map(InputLevel::from) + .collect_vec(), + splits: pb_compact_task + .splits + .iter() + .map(|pb_keyrange| KeyRange { + left: pb_keyrange.left.clone().into(), + right: pb_keyrange.right.clone().into(), + right_exclusive: pb_keyrange.right_exclusive, + }) + .collect_vec(), + watermark: pb_compact_task.watermark, + sorted_output_ssts: pb_compact_task + .sorted_output_ssts + .iter() + .map(SstableInfo::from) + .collect_vec(), + task_id: pb_compact_task.task_id, + target_level: pb_compact_task.target_level, + gc_delete_keys: pb_compact_task.gc_delete_keys, + base_level: pb_compact_task.base_level, + task_status: TaskStatus::try_from(pb_compact_task.task_status).unwrap(), + compaction_group_id: pb_compact_task.compaction_group_id, + existing_table_ids: pb_compact_task.existing_table_ids.clone(), + compression_algorithm: pb_compact_task.compression_algorithm, + target_file_size: pb_compact_task.target_file_size, + compaction_filter_mask: pb_compact_task.compaction_filter_mask, + table_options: pb_compact_task.table_options.clone(), + current_epoch_time: pb_compact_task.current_epoch_time, + target_sub_level_id: pb_compact_task.target_sub_level_id, + task_type: PbTaskType::try_from(pb_compact_task.task_type).unwrap(), + split_by_state_table: pb_compact_task.split_by_state_table, + split_weight_by_vnode: pb_compact_task.split_weight_by_vnode, + table_vnode_partition: pb_compact_task.table_vnode_partition.clone(), + table_watermarks: pb_compact_task + .table_watermarks + .iter() + .map(|(table_id, pb_table_watermark)| { + (*table_id, TableWatermarks::from(pb_table_watermark)) + }) + .collect(), + table_schemas: pb_compact_task.table_schemas.clone(), + max_sub_compaction: pb_compact_task.max_sub_compaction, + } + } +} + +impl From for PbCompactTask { + #[expect(deprecated)] + fn from(compact_task: CompactTask) -> Self { + Self { + input_ssts: compact_task + .input_ssts + .into_iter() + .map(|input_level| input_level.into()) + .collect_vec(), + splits: compact_task + .splits + .into_iter() + .map(|keyrange| PbKeyRange { + left: keyrange.left.into(), + right: keyrange.right.into(), + right_exclusive: keyrange.right_exclusive, + }) + .collect_vec(), + watermark: compact_task.watermark, + sorted_output_ssts: compact_task + .sorted_output_ssts + .into_iter() + .map(|sst| sst.into()) + .collect_vec(), + task_id: compact_task.task_id, + target_level: compact_task.target_level, + gc_delete_keys: compact_task.gc_delete_keys, + base_level: compact_task.base_level, + task_status: compact_task.task_status.into(), + compaction_group_id: compact_task.compaction_group_id, + existing_table_ids: compact_task.existing_table_ids.clone(), + compression_algorithm: compact_task.compression_algorithm, + target_file_size: compact_task.target_file_size, + compaction_filter_mask: compact_task.compaction_filter_mask, + table_options: compact_task.table_options.clone(), + current_epoch_time: compact_task.current_epoch_time, + target_sub_level_id: compact_task.target_sub_level_id, + task_type: compact_task.task_type.into(), + split_weight_by_vnode: compact_task.split_weight_by_vnode, + table_vnode_partition: compact_task.table_vnode_partition.clone(), + table_watermarks: compact_task + .table_watermarks + .into_iter() + .map(|(table_id, table_watermark)| (table_id, table_watermark.into())) + .collect(), + split_by_state_table: compact_task.split_by_state_table, + table_schemas: compact_task.table_schemas.clone(), + max_sub_compaction: compact_task.max_sub_compaction, + } + } +} + +impl From<&CompactTask> for PbCompactTask { + #[expect(deprecated)] + fn from(compact_task: &CompactTask) -> Self { + Self { + input_ssts: compact_task + .input_ssts + .iter() + .map(|input_level| input_level.into()) + .collect_vec(), + splits: compact_task + .splits + .iter() + .map(|keyrange| PbKeyRange { + left: keyrange.left.to_vec(), + right: keyrange.right.to_vec(), + right_exclusive: keyrange.right_exclusive, + }) + .collect_vec(), + watermark: compact_task.watermark, + sorted_output_ssts: compact_task + .sorted_output_ssts + .iter() + .map(|sst| sst.into()) + .collect_vec(), + task_id: compact_task.task_id, + target_level: compact_task.target_level, + gc_delete_keys: compact_task.gc_delete_keys, + base_level: compact_task.base_level, + task_status: compact_task.task_status.into(), + compaction_group_id: compact_task.compaction_group_id, + existing_table_ids: compact_task.existing_table_ids.clone(), + compression_algorithm: compact_task.compression_algorithm, + target_file_size: compact_task.target_file_size, + compaction_filter_mask: compact_task.compaction_filter_mask, + table_options: compact_task.table_options.clone(), + current_epoch_time: compact_task.current_epoch_time, + target_sub_level_id: compact_task.target_sub_level_id, + task_type: compact_task.task_type.into(), + split_weight_by_vnode: compact_task.split_weight_by_vnode, + table_vnode_partition: compact_task.table_vnode_partition.clone(), + table_watermarks: compact_task + .table_watermarks + .iter() + .map(|(table_id, table_watermark)| (*table_id, table_watermark.into())) + .collect(), + split_by_state_table: compact_task.split_by_state_table, + table_schemas: compact_task.table_schemas.clone(), + max_sub_compaction: compact_task.max_sub_compaction, + } + } +} + +#[derive(Clone, PartialEq, Default)] +pub struct ValidationTask { + pub sst_infos: Vec, + pub sst_id_to_worker_id: HashMap, + pub epoch: u64, +} + +impl From for ValidationTask { + fn from(pb_validation_task: PbValidationTask) -> Self { + Self { + sst_infos: pb_validation_task + .sst_infos + .into_iter() + .map(SstableInfo::from) + .collect_vec(), + sst_id_to_worker_id: pb_validation_task.sst_id_to_worker_id.clone(), + epoch: pb_validation_task.epoch, + } + } +} + +impl From for PbValidationTask { + fn from(validation_task: ValidationTask) -> Self { + Self { + sst_infos: validation_task + .sst_infos + .into_iter() + .map(|sst| sst.into()) + .collect_vec(), + sst_id_to_worker_id: validation_task.sst_id_to_worker_id.clone(), + epoch: validation_task.epoch, + } + } +} + +impl ValidationTask { + pub fn estimated_encode_len(&self) -> usize { + self.sst_infos + .iter() + .map(|sst| sst.estimated_encode_len()) + .sum::() + + self.sst_id_to_worker_id.len() * (size_of::() + size_of::()) + + size_of::() + } +} + +#[derive(Clone, PartialEq, Default, Debug)] +pub struct ReportTask { + pub table_stats_change: HashMap, + pub task_id: u64, + pub task_status: TaskStatus, + pub sorted_output_ssts: Vec, +} + +impl From for ReportTask { + fn from(value: PbReportTask) -> Self { + Self { + table_stats_change: value.table_stats_change.clone(), + task_id: value.task_id, + task_status: PbTaskStatus::try_from(value.task_status).unwrap(), + sorted_output_ssts: value + .sorted_output_ssts + .into_iter() + .map(SstableInfo::from) + .collect_vec(), + } + } +} + +impl From for PbReportTask { + fn from(value: ReportTask) -> Self { + Self { + table_stats_change: value.table_stats_change.clone(), + task_id: value.task_id, + task_status: value.task_status.into(), + sorted_output_ssts: value + .sorted_output_ssts + .into_iter() + .map(|sst| sst.into()) + .collect_vec(), + } + } +} 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 4e3a63c87ec9..b98e2d1abd78 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 @@ -19,17 +19,11 @@ use std::sync::Arc; use bytes::Bytes; use itertools::Itertools; -use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::TableId; use risingwave_common::hash::VnodeBitmapExt; -use risingwave_pb::hummock::group_delta::DeltaType; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::hummock_version_delta::GroupDeltas; -use risingwave_pb::hummock::table_watermarks::PbEpochNewWatermarks; use risingwave_pb::hummock::{ CompactionConfig, CompatibilityVersion, GroupConstruct, GroupDestroy, GroupMetaChange, - GroupTableChange, Level, LevelType, OverlappingLevel, PbLevelType, PbTableWatermarks, - SstableInfo, + GroupTableChange, PbLevelType, }; use tracing::warn; @@ -37,11 +31,12 @@ use super::StateTableId; use crate::change_log::TableChangeLog; use crate::compaction_group::StaticCompactionGroupId; use crate::key_range::KeyRangeCommon; -use crate::prost_key_range::KeyRangeExt; -use crate::table_watermark::{ - ReadTableWatermark, TableWatermarks, VnodeWatermark, WatermarkDirection, +use crate::level::{Level, Levels, OverlappingLevel}; +use crate::sstable_info::SstableInfo; +use crate::table_watermark::{ReadTableWatermark, TableWatermarks}; +use crate::version::{ + GroupDelta, GroupDeltas, HummockVersion, HummockVersionDelta, HummockVersionStateTableInfo, }; -use crate::version::{HummockVersion, HummockVersionDelta, HummockVersionStateTableInfo}; use crate::{can_concat, CompactionGroupId, HummockSstableId, HummockSstableObjectId}; pub struct GroupDeltasSummary { @@ -70,8 +65,8 @@ pub fn summarize_group_deltas(group_deltas: &GroupDeltas) -> GroupDeltasSummary let mut new_vnode_partition_count = 0; for group_delta in &group_deltas.group_deltas { - match group_delta.get_delta_type().unwrap() { - DeltaType::IntraLevel(intra_level) => { + match group_delta { + GroupDelta::IntraLevel(intra_level) => { if !intra_level.removed_table_ids.is_empty() { delete_sst_levels.push(intra_level.level_idx); delete_sst_ids_set.extend(intra_level.removed_table_ids.iter().clone()); @@ -83,18 +78,18 @@ pub fn summarize_group_deltas(group_deltas: &GroupDeltas) -> GroupDeltasSummary } new_vnode_partition_count = intra_level.vnode_partition_count; } - DeltaType::GroupConstruct(construct_delta) => { + GroupDelta::GroupConstruct(construct_delta) => { assert!(group_construct.is_none()); group_construct = Some(construct_delta.clone()); } - DeltaType::GroupDestroy(destroy_delta) => { + GroupDelta::GroupDestroy(destroy_delta) => { assert!(group_destroy.is_none()); group_destroy = Some(destroy_delta.clone()); } - DeltaType::GroupMetaChange(meta_delta) => { + GroupDelta::GroupMetaChange(meta_delta) => { group_meta_changes.push(meta_delta.clone()); } - DeltaType::GroupTableChange(meta_delta) => { + GroupDelta::GroupTableChange(meta_delta) => { group_table_change = Some(meta_delta.clone()); } } @@ -252,7 +247,7 @@ impl HummockVersion { pub fn safe_epoch_table_watermarks( &self, existing_table_ids: &[u32], - ) -> BTreeMap { + ) -> BTreeMap { safe_epoch_table_watermarks_impl( &self.table_watermarks, &self.state_table_info, @@ -265,11 +260,11 @@ pub fn safe_epoch_table_watermarks_impl( table_watermarks: &HashMap>, state_table_info: &HummockVersionStateTableInfo, existing_table_ids: &[u32], -) -> BTreeMap { +) -> BTreeMap { fn extract_single_table_watermark( table_watermarks: &TableWatermarks, safe_epoch: u64, - ) -> Option { + ) -> Option { if let Some((first_epoch, first_epoch_watermark)) = table_watermarks.watermarks.first() { assert!( *first_epoch >= safe_epoch, @@ -278,15 +273,9 @@ pub fn safe_epoch_table_watermarks_impl( safe_epoch ); if *first_epoch == safe_epoch { - Some(PbTableWatermarks { - epoch_watermarks: vec![PbEpochNewWatermarks { - watermarks: first_epoch_watermark - .iter() - .map(VnodeWatermark::to_protobuf) - .collect(), - epoch: *first_epoch, - }], - is_ascending: table_watermarks.direction.is_ascending(), + Some(TableWatermarks { + watermarks: vec![(*first_epoch, first_epoch_watermark.clone())], + direction: table_watermarks.direction, }) } else { None @@ -317,24 +306,17 @@ pub fn safe_epoch_table_watermarks_impl( } pub fn safe_epoch_read_table_watermarks_impl( - safe_epoch_watermarks: &BTreeMap, + safe_epoch_watermarks: &BTreeMap, ) -> BTreeMap { safe_epoch_watermarks .iter() .map(|(table_id, watermarks)| { - assert_eq!(watermarks.epoch_watermarks.len(), 1); - let vnode_watermarks = &watermarks - .epoch_watermarks - .first() - .expect("should exist") - .watermarks; + assert_eq!(watermarks.watermarks.len(), 1); + let vnode_watermarks = &watermarks.watermarks.first().expect("should exist").1; let mut vnode_watermark_map = BTreeMap::new(); - for vnode_watermark in vnode_watermarks { - let watermark = Bytes::copy_from_slice(&vnode_watermark.watermark); - for vnode in - Bitmap::from(vnode_watermark.vnode_bitmap.as_ref().expect("should exist")) - .iter_vnodes() - { + for vnode_watermark in vnode_watermarks.iter() { + let watermark = Bytes::copy_from_slice(vnode_watermark.watermark()); + for vnode in vnode_watermark.vnode_bitmap().iter_vnodes() { assert!( vnode_watermark_map .insert(vnode, watermark.clone()) @@ -347,11 +329,7 @@ pub fn safe_epoch_read_table_watermarks_impl( ( TableId::from(*table_id), ReadTableWatermark { - direction: if watermarks.is_ascending { - WatermarkDirection::Ascending - } else { - WatermarkDirection::Descending - }, + direction: watermarks.direction, vnode_watermarks: vnode_watermark_map, }, ) @@ -371,12 +349,12 @@ impl HummockVersion { parent_levels .l0 .iter() - .flat_map(|l0| l0.get_sub_levels()) - .chain(parent_levels.get_levels().iter()) - .flat_map(|level| level.get_table_infos()) + .flat_map(|l0| &l0.sub_levels) + .chain(parent_levels.levels.iter()) + .flat_map(|level| &level.table_infos) .map(|sst_info| { // `sst_info.table_ids` will never be empty. - for table_id in sst_info.get_table_ids() { + for table_id in &sst_info.table_ids { if member_table_ids.contains(table_id) { return 2; } @@ -448,7 +426,7 @@ impl HummockVersion { insert_new_sub_level( target_l0, sub_level.sub_level_id, - sub_level.level_type(), + sub_level.level_type, insert_table_infos, Some(idx), ); @@ -470,11 +448,9 @@ impl HummockVersion { cur_levels.levels[idx] .table_infos .extend(insert_table_infos); - cur_levels.levels[idx].table_infos.sort_by(|sst1, sst2| { - let a = sst1.key_range.as_ref().unwrap(); - let b = sst2.key_range.as_ref().unwrap(); - a.compare(b) - }); + cur_levels.levels[idx] + .table_infos + .sort_by(|sst1, sst2| sst1.key_range.cmp(&sst2.key_range)); assert!(can_concat(&cur_levels.levels[idx].table_infos)); level .table_infos @@ -505,7 +481,7 @@ impl HummockVersion { if !group_deltas .group_deltas .iter() - .all(|delta| matches!(delta.get_delta_type().unwrap(), DeltaType::IntraLevel(..))) + .all(|delta| matches!(delta, GroupDelta::IntraLevel(_))) { continue; } @@ -514,18 +490,21 @@ impl HummockVersion { // current `hummock::manager::gen_version_delta` implementation. Better refactor the // struct to reduce conventions. for group_delta in &group_deltas.group_deltas { - if let DeltaType::IntraLevel(delta) = group_delta.get_delta_type().unwrap() { - if !delta.inserted_table_infos.is_empty() { - info.insert_sst_level = delta.level_idx; + if let GroupDelta::IntraLevel(intra_level) = group_delta { + if !intra_level.inserted_table_infos.is_empty() { + info.insert_sst_level = intra_level.level_idx; info.insert_sst_infos - .extend(delta.inserted_table_infos.iter().cloned()); + .extend(intra_level.inserted_table_infos.iter().cloned()); } - if !delta.removed_table_ids.is_empty() { - for id in &delta.removed_table_ids { - if delta.level_idx == 0 { + if !intra_level.removed_table_ids.is_empty() { + for id in &intra_level.removed_table_ids { + if intra_level.level_idx == 0 { removed_l0_ssts.insert(*id); } else { - removed_ssts.entry(delta.level_idx).or_default().insert(*id); + removed_ssts + .entry(intra_level.level_idx) + .or_default() + .insert(*id); } } } @@ -533,14 +512,14 @@ impl HummockVersion { } let group = self.levels.get(group_id).unwrap(); - for l0_sub_level in &group.get_level0().sub_levels { + for l0_sub_level in &group.level0().sub_levels { for sst_info in &l0_sub_level.table_infos { if removed_l0_ssts.remove(&sst_info.sst_id) { info.delete_sst_object_ids.push(sst_info.object_id); } } } - for level in group.get_levels() { + 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 { if removed_level_ssts.remove(&sst_info.sst_id) { @@ -696,7 +675,7 @@ impl HummockVersion { insert_new_sub_level( levels.l0.as_mut().unwrap(), insert_sub_level_id, - LevelType::Overlapping, + PbLevelType::Overlapping, insert_table_infos, None, ); @@ -830,7 +809,7 @@ impl HummockVersion { if table_info.sst_id == table_info.object_id { continue; } - let object_id = table_info.get_object_id(); + let object_id = table_info.object_id; let entry: &mut BranchedSstInfo = ret.entry(object_id).or_default(); entry .entry(*compaction_group_id) @@ -845,34 +824,6 @@ impl HummockVersion { #[easy_ext::ext(HummockLevelsExt)] impl Levels { - pub fn get_level0(&self) -> &OverlappingLevel { - self.l0.as_ref().unwrap() - } - - pub fn get_level(&self, level_idx: usize) -> &Level { - &self.levels[level_idx - 1] - } - - pub fn get_level_mut(&mut self, level_idx: usize) -> &mut Level { - &mut self.levels[level_idx - 1] - } - - pub fn is_last_level(&self, level_idx: u32) -> bool { - self.levels - .last() - .as_ref() - .map_or(false, |level| level.level_idx == level_idx) - } - - pub fn count_ssts(&self) -> usize { - self.get_level0() - .get_sub_levels() - .iter() - .chain(self.get_levels().iter()) - .map(|level| level.get_table_infos().len()) - .sum() - } - pub fn apply_compact_ssts( &mut self, summary: GroupDeltasSummary, @@ -1015,7 +966,7 @@ pub fn build_initial_compaction_group_levels( for l in 0..compaction_config.get_max_level() { levels.push(Level { level_idx: (l + 1) as u32, - level_type: LevelType::Nonoverlapping as i32, + level_type: PbLevelType::Nonoverlapping, table_infos: vec![], total_file_size: 0, sub_level_id: 0, @@ -1096,16 +1047,16 @@ pub fn get_compaction_group_ssts( level .table_infos .iter() - .map(|table_info| (table_info.get_object_id(), table_info.get_sst_id())) + .map(|table_info| (table_info.object_id, table_info.sst_id)) }) } pub fn new_sub_level( sub_level_id: u64, - level_type: LevelType, + level_type: PbLevelType, table_infos: Vec, ) -> Level { - if level_type == LevelType::Nonoverlapping { + if level_type == PbLevelType::Nonoverlapping { debug_assert!( can_concat(&table_infos), "sst of non-overlapping level is not concat-able: {:?}", @@ -1119,7 +1070,7 @@ pub fn new_sub_level( .sum(); Level { level_idx: 0, - level_type: level_type as i32, + level_type, table_infos, total_file_size, sub_level_id, @@ -1142,14 +1093,10 @@ pub fn add_ssts_to_sub_level( l0.sub_levels[sub_level_idx] .table_infos .extend(insert_table_infos); - if l0.sub_levels[sub_level_idx].level_type == LevelType::Nonoverlapping as i32 { + if l0.sub_levels[sub_level_idx].level_type == PbLevelType::Nonoverlapping { l0.sub_levels[sub_level_idx] .table_infos - .sort_by(|sst1, sst2| { - let a = sst1.key_range.as_ref().unwrap(); - let b = sst2.key_range.as_ref().unwrap(); - a.compare(b) - }); + .sort_by(|sst1, sst2| sst1.key_range.cmp(&sst2.key_range)); assert!( can_concat(&l0.sub_levels[sub_level_idx].table_infos), "sstable ids: {:?}", @@ -1166,7 +1113,7 @@ pub fn add_ssts_to_sub_level( pub fn insert_new_sub_level( l0: &mut OverlappingLevel, insert_sub_level_id: u64, - level_type: LevelType, + level_type: PbLevelType, insert_table_infos: Vec, sub_level_insert_hint: Option, ) { @@ -1191,11 +1138,11 @@ pub fn insert_new_sub_level( { if insert_pos > 0 { if let Some(smaller_level) = l0.sub_levels.get(insert_pos - 1) { - debug_assert!(smaller_level.get_sub_level_id() < insert_sub_level_id); + debug_assert!(smaller_level.sub_level_id < insert_sub_level_id); } } if let Some(larger_level) = l0.sub_levels.get(insert_pos) { - debug_assert!(larger_level.get_sub_level_id() > insert_sub_level_id); + debug_assert!(larger_level.sub_level_id > insert_sub_level_id); } } // All files will be committed in one new Overlapping sub-level and become @@ -1240,13 +1187,11 @@ fn level_insert_ssts(operand: &mut Level, insert_table_infos: Vec) .map(|sst| sst.uncompressed_file_size) .sum::(); operand.table_infos.extend(insert_table_infos); - operand.table_infos.sort_by(|sst1, sst2| { - let a = sst1.key_range.as_ref().unwrap(); - let b = sst2.key_range.as_ref().unwrap(); - a.compare(b) - }); - if operand.level_type == LevelType::Overlapping as i32 { - operand.level_type = LevelType::Nonoverlapping as i32; + operand + .table_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), @@ -1264,16 +1209,11 @@ pub fn object_size_map(version: &HummockVersion) -> HashMap Vec { if level.table_infos.is_empty() { res.push(format!("{}: empty level", level_identifier)); } - } else if level.level_type() != PbLevelType::Nonoverlapping { + } else if level.level_type != PbLevelType::Nonoverlapping { // Ensure non-L0 level is non-overlapping level res.push(format!( "{}: level type {:?} is not non-overlapping", - level_identifier, - level.level_type() + level_identifier, level.level_type )); } @@ -1341,13 +1280,11 @@ pub fn validate_version(version: &HummockVersion) -> Vec { } // Ensure SSTs in non-overlapping level have non-overlapping key range - if level.level_type() == PbLevelType::Nonoverlapping { + if level.level_type == PbLevelType::Nonoverlapping { if let Some(prev) = prev_table_info.take() { if prev .key_range - .as_ref() - .unwrap() - .compare_right_with(&table_info.key_range.as_ref().unwrap().left) + .compare_right_with(&table_info.key_range.left) != Ordering::Less { res.push(format!( @@ -1399,16 +1336,14 @@ pub fn split_sst(sst_info: &mut SstableInfo, new_sst_id: &mut u64) -> SstableInf mod tests { use std::collections::HashMap; - use risingwave_pb::hummock::group_delta::DeltaType; - use risingwave_pb::hummock::hummock_version::Levels; - use risingwave_pb::hummock::hummock_version_delta::GroupDeltas; - use risingwave_pb::hummock::{ - CompactionConfig, GroupConstruct, GroupDelta, GroupDestroy, IntraLevelDelta, Level, - LevelType, OverlappingLevel, SstableInfo, - }; + use risingwave_pb::hummock::{CompactionConfig, GroupConstruct, GroupDestroy, LevelType}; use crate::compaction_group::hummock_version_ext::build_initial_compaction_group_levels; - use crate::version::{HummockVersion, HummockVersionDelta}; + use crate::level::{Level, Levels, OverlappingLevel}; + use crate::sstable_info::SstableInfo; + use crate::version::{ + GroupDelta, GroupDeltas, HummockVersion, HummockVersionDelta, IntraLevelDelta, + }; #[test] fn test_get_sst_object_ids() { @@ -1491,39 +1426,35 @@ mod tests { ( 2, GroupDeltas { - group_deltas: vec![GroupDelta { - delta_type: Some(DeltaType::GroupConstruct(GroupConstruct { - group_config: Some(CompactionConfig { - max_level: 6, - ..Default::default() - }), + group_deltas: vec![GroupDelta::GroupConstruct(GroupConstruct { + group_config: Some(CompactionConfig { + max_level: 6, ..Default::default() - })), - }], + }), + ..Default::default() + })], }, ), ( 0, GroupDeltas { - group_deltas: vec![GroupDelta { - delta_type: Some(DeltaType::GroupDestroy(GroupDestroy {})), - }], + group_deltas: vec![GroupDelta::GroupDestroy(GroupDestroy {})], }, ), ( 1, GroupDeltas { - group_deltas: vec![GroupDelta { - delta_type: Some(DeltaType::IntraLevel(IntraLevelDelta { - level_idx: 1, - inserted_table_infos: vec![SstableInfo { - object_id: 1, - sst_id: 1, - ..Default::default() - }], + group_deltas: vec![GroupDelta::IntraLevel(IntraLevelDelta::new( + 1, + 0, + vec![], + vec![SstableInfo { + object_id: 1, + sst_id: 1, ..Default::default() - })), - }], + }], + 0, + ))], }, ), ]); @@ -1539,7 +1470,7 @@ mod tests { ); cg1.levels[0] = Level { level_idx: 1, - level_type: LevelType::Nonoverlapping as i32, + level_type: LevelType::Nonoverlapping, table_infos: vec![SstableInfo { object_id: 1, sst_id: 1, diff --git a/src/storage/hummock_sdk/src/level.rs b/src/storage/hummock_sdk/src/level.rs new file mode 100644 index 000000000000..d81e153ebd91 --- /dev/null +++ b/src/storage/hummock_sdk/src/level.rs @@ -0,0 +1,406 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::mem::size_of; + +use itertools::Itertools; +use risingwave_pb::hummock::hummock_version::PbLevels; +use risingwave_pb::hummock::{ + PbInputLevel, PbLevel, PbLevelType, PbOverlappingLevel, PbSstableInfo, +}; + +use crate::sstable_info::SstableInfo; + +#[derive(Debug, Clone, PartialEq, Default)] +pub struct OverlappingLevel { + pub sub_levels: Vec, + pub total_file_size: u64, + pub uncompressed_file_size: u64, +} + +impl From<&PbOverlappingLevel> for OverlappingLevel { + fn from(pb_overlapping_level: &PbOverlappingLevel) -> Self { + Self { + sub_levels: pb_overlapping_level + .sub_levels + .iter() + .map(Level::from) + .collect_vec(), + total_file_size: pb_overlapping_level.total_file_size, + uncompressed_file_size: pb_overlapping_level.uncompressed_file_size, + } + } +} + +impl From<&OverlappingLevel> for PbOverlappingLevel { + fn from(overlapping_level: &OverlappingLevel) -> Self { + Self { + sub_levels: overlapping_level + .sub_levels + .iter() + .map(|pb_level| pb_level.into()) + .collect_vec(), + total_file_size: overlapping_level.total_file_size, + uncompressed_file_size: overlapping_level.uncompressed_file_size, + } + } +} + +impl From for PbOverlappingLevel { + fn from(overlapping_level: OverlappingLevel) -> Self { + Self { + sub_levels: overlapping_level + .sub_levels + .into_iter() + .map(|pb_level| pb_level.into()) + .collect_vec(), + total_file_size: overlapping_level.total_file_size, + uncompressed_file_size: overlapping_level.uncompressed_file_size, + } + } +} + +impl From for OverlappingLevel { + fn from(pb_overlapping_level: PbOverlappingLevel) -> Self { + Self { + sub_levels: pb_overlapping_level + .sub_levels + .into_iter() + .map(Level::from) + .collect_vec(), + total_file_size: pb_overlapping_level.total_file_size, + uncompressed_file_size: pb_overlapping_level.uncompressed_file_size, + } + } +} + +impl OverlappingLevel { + pub fn estimated_encode_len(&self) -> usize { + self.sub_levels + .iter() + .map(|level| level.estimated_encode_len()) + .sum::() + + size_of::() + + size_of::() + } +} + +#[derive(Debug, Clone, PartialEq, Default)] +pub struct Level { + pub level_idx: u32, + pub level_type: PbLevelType, + pub table_infos: Vec, + pub total_file_size: u64, + pub sub_level_id: u64, + pub uncompressed_file_size: u64, + pub vnode_partition_count: u32, +} + +impl From<&PbLevel> for Level { + fn from(pb_level: &PbLevel) -> Self { + 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(SstableInfo::from) + .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, + vnode_partition_count: pb_level.vnode_partition_count, + } + } +} + +impl From<&Level> for PbLevel { + fn from(level: &Level) -> Self { + Self { + level_idx: level.level_idx, + level_type: level.level_type.into(), + table_infos: level + .table_infos + .iter() + .map(PbSstableInfo::from) + .collect_vec(), + total_file_size: level.total_file_size, + sub_level_id: level.sub_level_id, + uncompressed_file_size: level.uncompressed_file_size, + vnode_partition_count: level.vnode_partition_count, + } + } +} + +impl From for PbLevel { + fn from(level: Level) -> Self { + Self { + level_idx: level.level_idx, + level_type: level.level_type.into(), + table_infos: level + .table_infos + .into_iter() + .map(PbSstableInfo::from) + .collect_vec(), + total_file_size: level.total_file_size, + sub_level_id: level.sub_level_id, + uncompressed_file_size: level.uncompressed_file_size, + vnode_partition_count: level.vnode_partition_count, + } + } +} + +impl From for Level { + fn from(pb_level: PbLevel) -> Self { + Self { + level_idx: pb_level.level_idx, + level_type: PbLevelType::try_from(pb_level.level_type).unwrap(), + table_infos: pb_level + .table_infos + .into_iter() + .map(SstableInfo::from) + .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, + vnode_partition_count: pb_level.vnode_partition_count, + } + } +} + +impl Level { + pub fn estimated_encode_len(&self) -> usize { + size_of::() + + size_of::() + + self + .table_infos + .iter() + .map(|sst| sst.estimated_encode_len()) + .sum::() + + size_of::() + + size_of::() + + size_of::() + + size_of::() + } +} + +#[derive(Debug, Clone, PartialEq, Default)] +pub struct Levels { + pub levels: Vec, + pub l0: Option, + pub group_id: u64, + pub parent_group_id: u64, + + #[deprecated] + pub member_table_ids: Vec, +} + +impl Levels { + pub fn level0(&self) -> &OverlappingLevel { + self.l0.as_ref().unwrap() + } + + pub fn get_level(&self, level_idx: usize) -> &Level { + &self.levels[level_idx - 1] + } + + pub fn get_level_mut(&mut self, level_idx: usize) -> &mut Level { + &mut self.levels[level_idx - 1] + } + + pub fn is_last_level(&self, level_idx: u32) -> bool { + self.levels + .last() + .as_ref() + .map_or(false, |level| level.level_idx == level_idx) + } + + pub fn count_ssts(&self) -> usize { + self.level0() + .sub_levels + .iter() + .chain(self.levels.iter()) + .map(|level| level.table_infos.len()) + .sum() + } +} + +impl Levels { + pub fn from_protobuf(pb_levels: &PbLevels) -> Self { + Self::from(pb_levels) + } + + pub fn to_protobuf(&self) -> PbLevels { + self.into() + } + + pub fn estimated_encode_len(&self) -> usize { + let mut basic = self + .levels + .iter() + .map(|level| level.estimated_encode_len()) + .sum::() + + size_of::() + + size_of::() + + size_of::(); + if let Some(l0) = self.l0.as_ref() { + basic += l0.estimated_encode_len(); + } + + basic + } +} + +impl From<&PbLevels> for Levels { + #[expect(deprecated)] + fn from(pb_levels: &PbLevels) -> Self { + Self { + l0: if pb_levels.l0.is_some() { + Some(OverlappingLevel::from(pb_levels.l0.as_ref().unwrap())) + } else { + None + }, + levels: pb_levels.levels.iter().map(Level::from).collect_vec(), + group_id: pb_levels.group_id, + parent_group_id: pb_levels.parent_group_id, + member_table_ids: pb_levels.member_table_ids.clone(), + } + } +} + +impl From<&Levels> for PbLevels { + #[expect(deprecated)] + fn from(levels: &Levels) -> Self { + Self { + l0: if levels.l0.is_some() { + Some(levels.l0.as_ref().unwrap().into()) + } else { + None + }, + levels: levels.levels.iter().map(PbLevel::from).collect_vec(), + group_id: levels.group_id, + parent_group_id: levels.parent_group_id, + member_table_ids: levels.member_table_ids.clone(), + } + } +} + +impl From for Levels { + #[expect(deprecated)] + fn from(pb_levels: PbLevels) -> Self { + Self { + l0: if pb_levels.l0.is_some() { + Some(OverlappingLevel::from(pb_levels.l0.unwrap())) + } else { + None + }, + levels: pb_levels.levels.into_iter().map(Level::from).collect_vec(), + group_id: pb_levels.group_id, + parent_group_id: pb_levels.parent_group_id, + member_table_ids: pb_levels.member_table_ids.clone(), + } + } +} + +impl From for PbLevels { + fn from(levels: Levels) -> Self { + #[expect(deprecated)] + Self { + l0: if levels.l0.is_some() { + Some(levels.l0.unwrap().into()) + } else { + None + }, + levels: levels.levels.into_iter().map(PbLevel::from).collect_vec(), + group_id: levels.group_id, + parent_group_id: levels.parent_group_id, + member_table_ids: levels.member_table_ids.clone(), + } + } +} + +#[derive(Clone, PartialEq, Default, Debug)] +pub struct InputLevel { + pub level_idx: u32, + pub level_type: PbLevelType, + pub table_infos: Vec, +} + +impl InputLevel { + pub fn estimated_encode_len(&self) -> usize { + size_of::() + + size_of::() + + self + .table_infos + .iter() + .map(|sst| sst.estimated_encode_len()) + .sum::() + } +} + +impl From for InputLevel { + fn from(pb_input_level: PbInputLevel) -> Self { + Self { + level_idx: pb_input_level.level_idx, + level_type: PbLevelType::try_from(pb_input_level.level_type).unwrap(), + table_infos: pb_input_level + .table_infos + .into_iter() + .map(SstableInfo::from) + .collect_vec(), + } + } +} + +impl From<&PbInputLevel> for InputLevel { + fn from(pb_input_level: &PbInputLevel) -> Self { + Self { + level_idx: pb_input_level.level_idx, + level_type: PbLevelType::try_from(pb_input_level.level_type).unwrap(), + table_infos: pb_input_level + .table_infos + .iter() + .map(SstableInfo::from) + .collect_vec(), + } + } +} + +impl From for PbInputLevel { + fn from(input_level: InputLevel) -> Self { + Self { + level_idx: input_level.level_idx, + level_type: input_level.level_type.into(), + table_infos: input_level + .table_infos + .into_iter() + .map(|sst| sst.into()) + .collect_vec(), + } + } +} + +impl From<&InputLevel> for PbInputLevel { + fn from(input_level: &InputLevel) -> Self { + Self { + level_idx: input_level.level_idx, + level_type: input_level.level_type.into(), + table_infos: input_level + .table_infos + .iter() + .map(|sst| sst.into()) + .collect_vec(), + } + } +} diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index 4b74a6e790c0..2ad827d8cae5 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -31,17 +31,20 @@ use std::collections::HashMap; pub use key_cmp::*; use risingwave_common::util::epoch::EPOCH_SPILL_TIME_MASK; use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; -use risingwave_pb::hummock::SstableInfo; +use sstable_info::SstableInfo; use crate::key_range::KeyRangeCommon; use crate::table_stats::TableStatsMap; pub mod change_log; pub mod compact; +pub mod compact_task; pub mod compaction_group; pub mod key; pub mod key_range; +pub mod level; pub mod prost_key_range; +pub mod sstable_info; pub mod table_stats; pub mod table_watermark; pub mod time_travel; @@ -210,9 +213,7 @@ pub fn can_concat(ssts: &[SstableInfo]) -> bool { for i in 1..len { if ssts[i - 1] .key_range - .as_ref() - .unwrap() - .compare_right_with(&ssts[i].key_range.as_ref().unwrap().left) + .compare_right_with(&ssts[i].key_range.left) != Ordering::Less { return false; diff --git a/src/storage/hummock_sdk/src/sstable_info.rs b/src/storage/hummock_sdk/src/sstable_info.rs new file mode 100644 index 000000000000..3fd7ff49686e --- /dev/null +++ b/src/storage/hummock_sdk/src/sstable_info.rs @@ -0,0 +1,180 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::mem::size_of; + +use risingwave_pb::hummock::{PbBloomFilterType, PbKeyRange, PbSstableInfo}; + +use crate::key_range::KeyRange; + +#[derive(Debug, PartialEq, Clone, Default)] +pub struct SstableInfo { + pub object_id: u64, + pub sst_id: u64, + pub key_range: KeyRange, + pub file_size: u64, + pub table_ids: Vec, + pub meta_offset: u64, + pub stale_key_count: u64, + pub total_key_count: u64, + pub min_epoch: u64, + pub max_epoch: u64, + pub uncompressed_file_size: u64, + pub range_tombstone_count: u64, + pub bloom_filter_kind: PbBloomFilterType, +} + +impl SstableInfo { + pub fn estimated_encode_len(&self) -> usize { + let mut basic = size_of::() // object_id + + size_of::() // sstable_id + + size_of::() // file_size + + self.table_ids.len() * size_of::() // table_ids + + size_of::() // meta_offset + + size_of::() // stale_key_count + + size_of::() // total_key_count + + size_of::() // min_epoch + + size_of::() // max_epoch + + size_of::() // uncompressed_file_size + + size_of::() // range_tombstone_count + + size_of::(); // bloom_filter_kind + basic += self.key_range.left.len() + self.key_range.right.len() + size_of::(); + + basic + } + + pub fn to_protobuf(&self) -> PbSstableInfo { + self.into() + } +} + +impl From for SstableInfo { + fn from(pb_sstable_info: PbSstableInfo) -> Self { + Self { + object_id: pb_sstable_info.object_id, + sst_id: pb_sstable_info.sst_id, + key_range: { + let pb_keyrange = pb_sstable_info.key_range.unwrap(); + KeyRange { + left: pb_keyrange.left.into(), + right: pb_keyrange.right.into(), + right_exclusive: pb_keyrange.right_exclusive, + } + }, + file_size: pb_sstable_info.file_size, + table_ids: pb_sstable_info.table_ids.clone(), + meta_offset: pb_sstable_info.meta_offset, + stale_key_count: pb_sstable_info.stale_key_count, + total_key_count: pb_sstable_info.total_key_count, + min_epoch: pb_sstable_info.min_epoch, + max_epoch: pb_sstable_info.max_epoch, + uncompressed_file_size: pb_sstable_info.uncompressed_file_size, + range_tombstone_count: pb_sstable_info.range_tombstone_count, + bloom_filter_kind: PbBloomFilterType::try_from(pb_sstable_info.bloom_filter_kind) + .unwrap(), + } + } +} + +impl From<&PbSstableInfo> for SstableInfo { + fn from(pb_sstable_info: &PbSstableInfo) -> Self { + Self { + object_id: pb_sstable_info.object_id, + sst_id: pb_sstable_info.sst_id, + key_range: { + let pb_keyrange = pb_sstable_info.key_range.as_ref().unwrap(); + KeyRange { + left: pb_keyrange.left.clone().into(), + right: pb_keyrange.right.clone().into(), + right_exclusive: pb_keyrange.right_exclusive, + } + }, + file_size: pb_sstable_info.file_size, + table_ids: pb_sstable_info.table_ids.clone(), + meta_offset: pb_sstable_info.meta_offset, + stale_key_count: pb_sstable_info.stale_key_count, + total_key_count: pb_sstable_info.total_key_count, + min_epoch: pb_sstable_info.min_epoch, + max_epoch: pb_sstable_info.max_epoch, + uncompressed_file_size: pb_sstable_info.uncompressed_file_size, + range_tombstone_count: pb_sstable_info.range_tombstone_count, + bloom_filter_kind: PbBloomFilterType::try_from(pb_sstable_info.bloom_filter_kind) + .unwrap(), + } + } +} + +impl From for PbSstableInfo { + fn from(sstable_info: SstableInfo) -> Self { + PbSstableInfo { + object_id: sstable_info.object_id, + sst_id: sstable_info.sst_id, + key_range: { + let keyrange = sstable_info.key_range; + let pb_key_range = PbKeyRange { + left: keyrange.left.into(), + right: keyrange.right.into(), + right_exclusive: keyrange.right_exclusive, + }; + Some(pb_key_range) + }, + + file_size: sstable_info.file_size, + table_ids: sstable_info.table_ids.clone(), + meta_offset: sstable_info.meta_offset, + stale_key_count: sstable_info.stale_key_count, + total_key_count: sstable_info.total_key_count, + min_epoch: sstable_info.min_epoch, + max_epoch: sstable_info.max_epoch, + uncompressed_file_size: sstable_info.uncompressed_file_size, + range_tombstone_count: sstable_info.range_tombstone_count, + bloom_filter_kind: sstable_info.bloom_filter_kind.into(), + } + } +} + +impl From<&SstableInfo> for PbSstableInfo { + fn from(sstable_info: &SstableInfo) -> Self { + PbSstableInfo { + object_id: sstable_info.object_id, + sst_id: sstable_info.sst_id, + key_range: { + let keyrange = &sstable_info.key_range; + let pb_key_range = PbKeyRange { + left: keyrange.left.to_vec(), + right: keyrange.right.to_vec(), + right_exclusive: keyrange.right_exclusive, + }; + Some(pb_key_range) + }, + + file_size: sstable_info.file_size, + table_ids: sstable_info.table_ids.clone(), + meta_offset: sstable_info.meta_offset, + stale_key_count: sstable_info.stale_key_count, + total_key_count: sstable_info.total_key_count, + min_epoch: sstable_info.min_epoch, + max_epoch: sstable_info.max_epoch, + uncompressed_file_size: sstable_info.uncompressed_file_size, + range_tombstone_count: sstable_info.range_tombstone_count, + bloom_filter_kind: sstable_info.bloom_filter_kind.into(), + } + } +} + +impl SstableInfo { + pub fn remove_key_range(&mut self) { + self.key_range = KeyRange::default(); + } +} diff --git a/src/storage/hummock_sdk/src/table_watermark.rs b/src/storage/hummock_sdk/src/table_watermark.rs index 51b45e70199d..2de3fd1b6942 100644 --- a/src/storage/hummock_sdk/src/table_watermark.rs +++ b/src/storage/hummock_sdk/src/table_watermark.rs @@ -320,26 +320,52 @@ impl VnodeWatermark { } } + pub fn vnode_bitmap(&self) -> &Bitmap { + &self.vnode_bitmap + } + + pub fn watermark(&self) -> &Bytes { + &self.watermark + } + pub fn to_protobuf(&self) -> PbVnodeWatermark { - PbVnodeWatermark { - watermark: self.watermark.to_vec(), - vnode_bitmap: Some(self.vnode_bitmap.to_protobuf()), + self.into() + } +} + +impl From for VnodeWatermark { + fn from(pb: PbVnodeWatermark) -> Self { + Self { + vnode_bitmap: Arc::new(Bitmap::from(pb.vnode_bitmap.as_ref().unwrap())), + watermark: Bytes::from(pb.watermark), } } +} - pub fn from_protobuf(pb: &PbVnodeWatermark) -> Self { +impl From<&PbVnodeWatermark> for VnodeWatermark { + fn from(pb: &PbVnodeWatermark) -> Self { Self { vnode_bitmap: Arc::new(Bitmap::from(pb.vnode_bitmap.as_ref().unwrap())), watermark: Bytes::from(pb.watermark.clone()), } } +} - pub fn vnode_bitmap(&self) -> &Bitmap { - &self.vnode_bitmap +impl From for PbVnodeWatermark { + fn from(watermark: VnodeWatermark) -> Self { + Self { + watermark: watermark.watermark.into(), + vnode_bitmap: Some(watermark.vnode_bitmap.to_protobuf()), + } } +} - pub fn watermark(&self) -> &Bytes { - &self.watermark +impl From<&VnodeWatermark> for PbVnodeWatermark { + fn from(watermark: &VnodeWatermark) -> Self { + Self { + watermark: watermark.watermark.to_vec(), + vnode_bitmap: Some(watermark.vnode_bitmap.to_protobuf()), + } } } @@ -362,38 +388,6 @@ impl TableWatermarks { } } - pub fn to_protobuf(&self) -> PbTableWatermarks { - PbTableWatermarks { - epoch_watermarks: self - .watermarks - .iter() - .map(|(epoch, watermarks)| PbEpochNewWatermarks { - watermarks: watermarks.iter().map(VnodeWatermark::to_protobuf).collect(), - epoch: *epoch, - }) - .collect(), - is_ascending: match self.direction { - WatermarkDirection::Ascending => true, - WatermarkDirection::Descending => false, - }, - } - } - - pub fn estimated_encode_len(&self) -> usize { - self.watermarks.len() * size_of::() - + self - .watermarks - .iter() - .map(|(_, watermarks)| { - watermarks - .iter() - .map(|watermark| watermark.estimated_size()) - .sum::() - }) - .sum::() - + size_of::() // for direction - } - pub fn add_new_epoch_watermarks( &mut self, epoch: HummockEpoch, @@ -417,7 +411,7 @@ impl TableWatermarks { let watermarks = epoch_watermark .watermarks .iter() - .map(VnodeWatermark::from_protobuf) + .map(VnodeWatermark::from) .collect_vec(); (epoch, Arc::from(watermarks)) }) @@ -580,6 +574,115 @@ impl TableWatermarks { } } +impl TableWatermarks { + pub fn estimated_encode_len(&self) -> usize { + self.watermarks.len() * size_of::() + + self + .watermarks + .iter() + .map(|(_, watermarks)| { + watermarks + .iter() + .map(|watermark| watermark.estimated_size()) + .sum::() + }) + .sum::() + + size_of::() // for direction + } + + pub fn to_protobuf(&self) -> PbTableWatermarks { + self.into() + } +} + +impl From<&PbTableWatermarks> for TableWatermarks { + fn from(pb: &PbTableWatermarks) -> Self { + Self { + watermarks: pb + .epoch_watermarks + .iter() + .map(|epoch_watermark| { + let epoch = epoch_watermark.epoch; + let watermarks = epoch_watermark + .watermarks + .iter() + .map(VnodeWatermark::from) + .collect(); + (epoch, watermarks) + }) + .collect(), + direction: if pb.is_ascending { + WatermarkDirection::Ascending + } else { + WatermarkDirection::Descending + }, + } + } +} + +impl From<&TableWatermarks> for PbTableWatermarks { + fn from(table_watermarks: &TableWatermarks) -> Self { + Self { + epoch_watermarks: table_watermarks + .watermarks + .iter() + .map(|(epoch, watermarks)| PbEpochNewWatermarks { + watermarks: watermarks.iter().map(|wm| wm.into()).collect(), + epoch: *epoch, + }) + .collect(), + is_ascending: match table_watermarks.direction { + WatermarkDirection::Ascending => true, + WatermarkDirection::Descending => false, + }, + } + } +} + +impl From for TableWatermarks { + fn from(pb: PbTableWatermarks) -> Self { + Self { + watermarks: pb + .epoch_watermarks + .into_iter() + .map(|epoch_watermark| { + let epoch = epoch_watermark.epoch; + let watermarks = epoch_watermark + .watermarks + .into_iter() + .map(VnodeWatermark::from) + .collect(); + (epoch, watermarks) + }) + .collect(), + direction: if pb.is_ascending { + WatermarkDirection::Ascending + } else { + WatermarkDirection::Descending + }, + } + } +} + +impl From for PbTableWatermarks { + fn from(table_watermarks: TableWatermarks) -> Self { + Self { + epoch_watermarks: table_watermarks + .watermarks + .into_iter() + .map(|(epoch, watermarks)| PbEpochNewWatermarks { + watermarks: watermarks.iter().map(PbVnodeWatermark::from).collect(), + epoch, + }) + .collect(), + is_ascending: match table_watermarks.direction { + WatermarkDirection::Ascending => true, + WatermarkDirection::Descending => false, + }, + } + } +} + #[cfg(test)] mod tests { use std::collections::Bound::Included; diff --git a/src/storage/hummock_sdk/src/time_travel.rs b/src/storage/hummock_sdk/src/time_travel.rs index 2a848c272e23..7da9197e54dd 100644 --- a/src/storage/hummock_sdk/src/time_travel.rs +++ b/src/storage/hummock_sdk/src/time_travel.rs @@ -16,17 +16,17 @@ use std::collections::{HashMap, HashSet}; use std::sync::Arc; use risingwave_common::catalog::TableId; -use risingwave_pb::hummock::group_delta::DeltaType; -use risingwave_pb::hummock::hummock_version::PbLevels; -use risingwave_pb::hummock::hummock_version_delta::{ChangeLogDelta, PbGroupDeltas}; -use risingwave_pb::hummock::{ - EpochNewChangeLog, PbGroupDelta, PbHummockVersion, PbHummockVersionDelta, PbIntraLevelDelta, - PbLevel, PbOverlappingLevel, PbSstableInfo, SstableInfo, StateTableInfoDelta, -}; +use risingwave_pb::hummock::hummock_version_delta::PbGroupDeltas; +use risingwave_pb::hummock::{PbHummockVersion, PbHummockVersionDelta, PbStateTableInfoDelta}; -use crate::change_log::TableChangeLog; +use crate::change_log::{ChangeLogDelta, EpochNewChangeLog, TableChangeLog}; +use crate::level::{Level, Levels, OverlappingLevel}; +use crate::sstable_info::SstableInfo; use crate::table_watermark::TableWatermarks; -use crate::version::{HummockVersion, HummockVersionDelta, HummockVersionStateTableInfo}; +use crate::version::{ + GroupDelta, GroupDeltas, HummockVersion, HummockVersionDelta, HummockVersionStateTableInfo, + IntraLevelDelta, +}; use crate::{CompactionGroupId, HummockSstableId}; /// [`IncompleteHummockVersion`] is incomplete because `SSTableInfo` only has the `sst_id` set in the following fields: @@ -35,7 +35,7 @@ use crate::{CompactionGroupId, HummockSstableId}; #[derive(Debug, Clone, PartialEq)] pub struct IncompleteHummockVersion { pub id: u64, - pub levels: HashMap, + pub levels: HashMap, pub max_committed_epoch: u64, safe_epoch: u64, pub table_watermarks: HashMap>, @@ -78,8 +78,8 @@ fn stripped_change_log_delta(origin: &ChangeLogDelta) -> ChangeLogDelta { } } -fn stripped_level(origin: &PbLevel) -> PbLevel { - PbLevel { +fn stripped_level(origin: &Level) -> Level { + Level { level_idx: origin.level_idx, level_type: origin.level_type, table_infos: origin @@ -96,7 +96,7 @@ fn stripped_level(origin: &PbLevel) -> PbLevel { pub fn refill_version( version: &mut HummockVersion, - sst_id_to_info: &HashMap, + sst_id_to_info: &HashMap, ) { for level in version.levels.values_mut().flat_map(|level| { level @@ -116,7 +116,7 @@ pub fn refill_version( } } -fn refill_level(level: &mut PbLevel, sst_id_to_info: &HashMap) { +fn refill_level(level: &mut Level, sst_id_to_info: &HashMap) { for s in &mut level.table_infos { refill_sstable_info(s, sst_id_to_info); } @@ -124,7 +124,7 @@ fn refill_level(level: &mut PbLevel, sst_id_to_info: &HashMap, + sst_id_to_info: &HashMap, ) { for c in &mut table_change_log.0 { for s in &mut c.old_value { @@ -138,8 +138,8 @@ fn refill_table_change_log( /// Caller should ensure `sst_id_to_info` includes an entry corresponding to `sstable_info`. fn refill_sstable_info( - sstable_info: &mut PbSstableInfo, - sst_id_to_info: &HashMap, + sstable_info: &mut SstableInfo, + sst_id_to_info: &HashMap, ) { *sstable_info = sst_id_to_info .get(&sstable_info.sst_id) @@ -147,8 +147,8 @@ fn refill_sstable_info( .clone(); } -fn stripped_l0(origin: &PbOverlappingLevel) -> PbOverlappingLevel { - PbOverlappingLevel { +fn stripped_l0(origin: &OverlappingLevel) -> OverlappingLevel { + OverlappingLevel { sub_levels: origin.sub_levels.iter().map(stripped_level).collect(), total_file_size: origin.total_file_size, uncompressed_file_size: origin.uncompressed_file_size, @@ -156,8 +156,8 @@ fn stripped_l0(origin: &PbOverlappingLevel) -> PbOverlappingLevel { } #[allow(deprecated)] -fn stripped_levels(origin: &PbLevels) -> PbLevels { - PbLevels { +fn stripped_levels(origin: &Levels) -> Levels { + Levels { levels: origin.levels.iter().map(stripped_level).collect(), l0: origin.l0.as_ref().map(stripped_l0), group_id: origin.group_id, @@ -166,8 +166,8 @@ fn stripped_levels(origin: &PbLevels) -> PbLevels { } } -fn stripped_intra_level_delta(origin: &PbIntraLevelDelta) -> PbIntraLevelDelta { - PbIntraLevelDelta { +fn stripped_intra_level_delta(origin: &IntraLevelDelta) -> IntraLevelDelta { + IntraLevelDelta { level_idx: origin.level_idx, l0_sub_level_id: origin.l0_sub_level_id, removed_table_ids: origin.removed_table_ids.clone(), @@ -180,21 +180,20 @@ fn stripped_intra_level_delta(origin: &PbIntraLevelDelta) -> PbIntraLevelDelta { } } -fn stripped_group_delta(origin: &PbGroupDelta) -> PbGroupDelta { - let delta_type = origin.delta_type.as_ref().map(|d| match d { - DeltaType::IntraLevel(l) => DeltaType::IntraLevel(stripped_intra_level_delta(l)), +fn stripped_group_delta(origin: &GroupDelta) -> GroupDelta { + match origin { + GroupDelta::IntraLevel(l) => GroupDelta::IntraLevel(stripped_intra_level_delta(l)), _ => panic!("time travel expects DeltaType::IntraLevel only"), - }); - PbGroupDelta { delta_type } + } } -fn stripped_group_deltas(origin: &PbGroupDeltas) -> PbGroupDeltas { +fn stripped_group_deltas(origin: &GroupDeltas) -> GroupDeltas { let group_deltas = origin .group_deltas .iter() .map(stripped_group_delta) .collect(); - PbGroupDeltas { group_deltas } + GroupDeltas { group_deltas } } /// `SStableInfo` will be stripped. @@ -243,7 +242,7 @@ impl IncompleteHummockVersion { levels: self .levels .iter() - .map(|(group_id, levels)| (*group_id as _, levels.clone())) + .map(|(group_id, levels)| (*group_id as _, levels.to_protobuf())) .collect(), max_committed_epoch: self.max_committed_epoch, safe_epoch: self.safe_epoch, @@ -276,7 +275,7 @@ pub struct IncompleteHummockVersionDelta { pub new_table_watermarks: HashMap, pub removed_table_ids: HashSet, pub change_log_delta: HashMap, - pub state_table_info_delta: HashMap, + pub state_table_info_delta: HashMap, } /// `SStableInfo` will be stripped. @@ -291,7 +290,7 @@ impl From<(&HummockVersionDelta, &HashSet)> for IncompleteHum .iter() .filter_map(|(cg_id, deltas)| { if select_group.contains(cg_id) { - Some((*cg_id, stripped_group_deltas(deltas))) + Some((*cg_id, stripped_group_deltas(deltas).to_protobuf())) } else { None } @@ -336,7 +335,7 @@ impl IncompleteHummockVersionDelta { change_log_delta: self .change_log_delta .iter() - .map(|(table_id, log_delta)| (table_id.table_id, log_delta.clone())) + .map(|(table_id, log_delta)| (table_id.table_id, log_delta.into())) .collect(), state_table_info_delta: self .state_table_info_delta diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 920c43df1df1..d38376a7fda0 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -18,22 +18,23 @@ use std::mem::{replace, size_of}; use std::ops::Deref; use std::sync::{Arc, LazyLock}; -use prost::Message; +use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::INVALID_EPOCH; -use risingwave_pb::hummock::group_delta::DeltaType; -use risingwave_pb::hummock::hummock_version::Levels as PbLevels; -use risingwave_pb::hummock::hummock_version_delta::{ChangeLogDelta, GroupDeltas as PbGroupDeltas}; +use risingwave_pb::hummock::group_delta::PbDeltaType; +use risingwave_pb::hummock::hummock_version_delta::PbGroupDeltas; use risingwave_pb::hummock::{ - CompactionConfig, HummockVersion as PbHummockVersion, - HummockVersionDelta as PbHummockVersionDelta, SstableInfo, StateTableInfo as PbStateTableInfo, - StateTableInfo, StateTableInfoDelta, + CompactionConfig, PbGroupConstruct, PbGroupDelta, PbGroupDestroy, PbGroupMetaChange, + PbGroupTableChange, PbHummockVersion, PbHummockVersionDelta, PbIntraLevelDelta, + PbStateTableInfo, StateTableInfo, StateTableInfoDelta, }; use tracing::warn; -use crate::change_log::TableChangeLog; +use crate::change_log::{ChangeLogDelta, TableChangeLog}; use crate::compaction_group::hummock_version_ext::build_initial_compaction_group_levels; use crate::compaction_group::StaticCompactionGroupId; +use crate::level::Levels; +use crate::sstable_info::SstableInfo; use crate::table_watermark::TableWatermarks; use crate::{CompactionGroupId, HummockSstableObjectId, HummockVersionId, FIRST_VERSION_ID}; @@ -205,7 +206,7 @@ impl HummockVersionStateTableInfo { #[derive(Debug, Clone, PartialEq)] pub struct HummockVersion { pub id: u64, - pub levels: HashMap, + pub levels: HashMap, pub max_committed_epoch: u64, safe_epoch: u64, pub table_watermarks: HashMap>, @@ -215,7 +216,7 @@ pub struct HummockVersion { impl Default for HummockVersion { fn default() -> Self { - HummockVersion::from_protobuf_inner(&PbHummockVersion::default()) + HummockVersion::from(&PbHummockVersion::default()) } } @@ -223,22 +224,45 @@ impl HummockVersion { /// Convert the `PbHummockVersion` received from rpc to `HummockVersion`. No need to /// maintain backward compatibility. pub fn from_rpc_protobuf(pb_version: &PbHummockVersion) -> Self { - Self::from_protobuf_inner(pb_version) + HummockVersion::from(pb_version) } /// Convert the `PbHummockVersion` deserialized from persisted state to `HummockVersion`. /// We should maintain backward compatibility. pub fn from_persisted_protobuf(pb_version: &PbHummockVersion) -> Self { - Self::from_protobuf_inner(pb_version) + HummockVersion::from(pb_version) } - fn from_protobuf_inner(pb_version: &PbHummockVersion) -> Self { + pub fn to_protobuf(&self) -> PbHummockVersion { + self.into() + } +} + +impl HummockVersion { + pub fn estimated_encode_len(&self) -> usize { + self.levels.len() * size_of::() + + self + .levels + .values() + .map(|level| level.estimated_encode_len()) + .sum::() + + self.table_watermarks.len() * size_of::() + + self + .table_watermarks + .values() + .map(|table_watermark| table_watermark.estimated_encode_len()) + .sum::() + } +} + +impl From<&PbHummockVersion> for HummockVersion { + fn from(pb_version: &PbHummockVersion) -> Self { Self { id: pb_version.id, levels: pb_version .levels .iter() - .map(|(group_id, levels)| (*group_id as CompactionGroupId, levels.clone())) + .map(|(group_id, levels)| (*group_id as CompactionGroupId, Levels::from(levels))) .collect(), max_committed_epoch: pb_version.max_committed_epoch, safe_epoch: pb_version.safe_epoch, @@ -248,7 +272,7 @@ impl HummockVersion { .map(|(table_id, table_watermark)| { ( TableId::new(*table_id), - Arc::new(TableWatermarks::from_protobuf(table_watermark)), + Arc::new(TableWatermarks::from(table_watermark)), ) }) .collect(), @@ -267,46 +291,61 @@ impl HummockVersion { ), } } +} - pub fn to_protobuf(&self) -> PbHummockVersion { - PbHummockVersion { - id: self.id, - levels: self +impl From<&HummockVersion> for PbHummockVersion { + fn from(version: &HummockVersion) -> Self { + Self { + id: version.id, + levels: version .levels .iter() - .map(|(group_id, levels)| (*group_id as _, levels.clone())) + .map(|(group_id, levels)| (*group_id as _, levels.into())) .collect(), - max_committed_epoch: self.max_committed_epoch, - safe_epoch: self.safe_epoch, - table_watermarks: self + max_committed_epoch: version.max_committed_epoch, + safe_epoch: version.safe_epoch, + table_watermarks: version .table_watermarks .iter() - .map(|(table_id, watermark)| (table_id.table_id, watermark.to_protobuf())) + .map(|(table_id, watermark)| (table_id.table_id, watermark.as_ref().into())) .collect(), - table_change_logs: self + table_change_logs: version .table_change_log .iter() .map(|(table_id, change_log)| (table_id.table_id, change_log.to_protobuf())) .collect(), - state_table_info: self.state_table_info.to_protobuf(), + state_table_info: version.state_table_info.to_protobuf(), } } +} - pub fn estimated_encode_len(&self) -> usize { - self.levels.len() * size_of::() - + self +impl From for PbHummockVersion { + fn from(version: HummockVersion) -> Self { + Self { + id: version.id, + levels: version .levels - .values() - .map(|level| level.encoded_len()) - .sum::() - + self.table_watermarks.len() * size_of::() - + self + .into_iter() + .map(|(group_id, levels)| (group_id as _, levels.into())) + .collect(), + max_committed_epoch: version.max_committed_epoch, + safe_epoch: version.safe_epoch, + table_watermarks: version .table_watermarks - .values() - .map(|table_watermark| table_watermark.estimated_encode_len()) - .sum::() + .into_iter() + .map(|(table_id, watermark)| (table_id.table_id, watermark.as_ref().into())) + .collect(), + table_change_logs: version + .table_change_log + .into_iter() + .map(|(table_id, change_log)| (table_id.table_id, change_log.to_protobuf())) + .collect(), + state_table_info: version.state_table_info.to_protobuf(), + } } +} +impl HummockVersion { pub fn next_version_id(&self) -> HummockVersionId { self.id + 1 } @@ -399,7 +438,7 @@ impl HummockVersion { pub struct HummockVersionDelta { pub id: u64, pub prev_id: u64, - pub group_deltas: HashMap, + pub group_deltas: HashMap, pub max_committed_epoch: u64, safe_epoch: u64, pub trivial_move: bool, @@ -411,7 +450,7 @@ pub struct HummockVersionDelta { impl Default for HummockVersionDelta { fn default() -> Self { - HummockVersionDelta::from_protobuf_inner(&PbHummockVersionDelta::default()) + HummockVersionDelta::from(&PbHummockVersionDelta::default()) } } @@ -419,88 +458,17 @@ impl HummockVersionDelta { /// Convert the `PbHummockVersionDelta` deserialized from persisted state to `HummockVersionDelta`. /// We should maintain backward compatibility. pub fn from_persisted_protobuf(delta: &PbHummockVersionDelta) -> Self { - Self::from_protobuf_inner(delta) + Self::from(delta) } /// Convert the `PbHummockVersionDelta` received from rpc to `HummockVersionDelta`. No need to /// maintain backward compatibility. pub fn from_rpc_protobuf(delta: &PbHummockVersionDelta) -> Self { - Self::from_protobuf_inner(delta) - } - - fn from_protobuf_inner(delta: &PbHummockVersionDelta) -> Self { - Self { - id: delta.id, - prev_id: delta.prev_id, - group_deltas: delta.group_deltas.clone(), - max_committed_epoch: delta.max_committed_epoch, - safe_epoch: delta.safe_epoch, - trivial_move: delta.trivial_move, - new_table_watermarks: delta - .new_table_watermarks - .iter() - .map(|(table_id, watermarks)| { - ( - TableId::new(*table_id), - TableWatermarks::from_protobuf(watermarks), - ) - }) - .collect(), - removed_table_ids: delta - .removed_table_ids - .iter() - .map(|table_id| TableId::new(*table_id)) - .collect(), - change_log_delta: delta - .change_log_delta - .iter() - .map(|(table_id, log_delta)| { - ( - TableId::new(*table_id), - ChangeLogDelta { - new_log: log_delta.new_log.clone(), - truncate_epoch: log_delta.truncate_epoch, - }, - ) - }) - .collect(), - state_table_info_delta: delta - .state_table_info_delta - .iter() - .map(|(table_id, delta)| (TableId::new(*table_id), delta.clone())) - .collect(), - } + Self::from(delta) } pub fn to_protobuf(&self) -> PbHummockVersionDelta { - PbHummockVersionDelta { - id: self.id, - prev_id: self.prev_id, - group_deltas: self.group_deltas.clone(), - max_committed_epoch: self.max_committed_epoch, - safe_epoch: self.safe_epoch, - trivial_move: self.trivial_move, - new_table_watermarks: self - .new_table_watermarks - .iter() - .map(|(table_id, watermarks)| (table_id.table_id, watermarks.to_protobuf())) - .collect(), - removed_table_ids: self - .removed_table_ids - .iter() - .map(|table_id| table_id.table_id) - .collect(), - change_log_delta: self - .change_log_delta - .iter() - .map(|(table_id, log_delta)| (table_id.table_id, log_delta.clone())) - .collect(), - state_table_info_delta: self - .state_table_info_delta - .iter() - .map(|(table_id, delta)| (table_id.table_id, delta.clone())) - .collect(), - } + self.into() } } @@ -514,17 +482,15 @@ impl HummockVersionDelta { .values() .flat_map(|group_deltas| { group_deltas.group_deltas.iter().flat_map(|group_delta| { - group_delta.delta_type.iter().flat_map(|delta_type| { - static EMPTY_VEC: Vec = Vec::new(); - let sst_slice = match delta_type { - DeltaType::IntraLevel(level_delta) => &level_delta.inserted_table_infos, - DeltaType::GroupConstruct(_) - | DeltaType::GroupDestroy(_) - | DeltaType::GroupMetaChange(_) - | DeltaType::GroupTableChange(_) => &EMPTY_VEC, - }; - sst_slice.iter().map(|sst| sst.object_id) - }) + static EMPTY_VEC: Vec = Vec::new(); + let sst_slice = match group_delta { + GroupDelta::IntraLevel(level_delta) => &level_delta.inserted_table_infos, + GroupDelta::GroupConstruct(_) + | GroupDelta::GroupDestroy(_) + | GroupDelta::GroupMetaChange(_) + | GroupDelta::GroupTableChange(_) => &EMPTY_VEC, + }; + sst_slice.iter().map(|sst| sst.object_id) }) }) .chain(self.change_log_delta.values().flat_map(|delta| { @@ -539,31 +505,28 @@ impl HummockVersionDelta { } pub fn newly_added_sst_ids(&self) -> HashSet { - self.group_deltas - .values() - .flat_map(|group_deltas| { - group_deltas.group_deltas.iter().flat_map(|group_delta| { - group_delta.delta_type.iter().flat_map(|delta_type| { - static EMPTY_VEC: Vec = Vec::new(); - let sst_slice = match delta_type { - DeltaType::IntraLevel(level_delta) => &level_delta.inserted_table_infos, - DeltaType::GroupConstruct(_) - | DeltaType::GroupDestroy(_) - | DeltaType::GroupMetaChange(_) - | DeltaType::GroupTableChange(_) => &EMPTY_VEC, - }; - sst_slice.iter().map(|sst| sst.sst_id) - }) - }) + let ssts_from_group_deltas = self.group_deltas.values().flat_map(|group_deltas| { + group_deltas.group_deltas.iter().flat_map(|group_delta| { + static EMPTY_VEC: Vec = Vec::new(); + let sst_slice = match group_delta { + GroupDelta::IntraLevel(level_delta) => &level_delta.inserted_table_infos, + GroupDelta::GroupConstruct(_) + | GroupDelta::GroupDestroy(_) + | GroupDelta::GroupMetaChange(_) + | GroupDelta::GroupTableChange(_) => &EMPTY_VEC, + }; + sst_slice.iter() }) - .chain(self.change_log_delta.values().flat_map(|delta| { - let new_log = delta.new_log.as_ref().unwrap(); - new_log - .new_value - .iter() - .map(|sst| sst.sst_id) - .chain(new_log.old_value.iter().map(|sst| sst.sst_id)) - })) + }); + + let ssts_from_change_log = self.change_log_delta.values().flat_map(|delta| { + let new_log = delta.new_log.as_ref().unwrap(); + new_log.new_value.iter().chain(new_log.old_value.iter()) + }); + + ssts_from_group_deltas + .chain(ssts_from_change_log) + .map(|sst| sst.object_id) .collect() } @@ -582,17 +545,15 @@ impl HummockVersionDelta { }) .flat_map(|group_deltas| { group_deltas.group_deltas.iter().flat_map(|group_delta| { - group_delta.delta_type.iter().flat_map(|delta_type| { - static EMPTY_VEC: Vec = Vec::new(); - let sst_slice = match delta_type { - DeltaType::IntraLevel(level_delta) => &level_delta.inserted_table_infos, - DeltaType::GroupConstruct(_) - | DeltaType::GroupDestroy(_) - | DeltaType::GroupMetaChange(_) - | DeltaType::GroupTableChange(_) => &EMPTY_VEC, - }; - sst_slice.iter() - }) + static EMPTY_VEC: Vec = Vec::new(); + let sst_slice = match group_delta { + GroupDelta::IntraLevel(level_delta) => &level_delta.inserted_table_infos, + GroupDelta::GroupConstruct(_) + | GroupDelta::GroupDestroy(_) + | GroupDelta::GroupMetaChange(_) + | GroupDelta::GroupTableChange(_) => &EMPTY_VEC, + }; + sst_slice.iter() }) }) .chain(self.change_log_delta.values().flat_map(|delta| { @@ -610,3 +571,437 @@ impl HummockVersionDelta { self.safe_epoch = safe_epoch; } } + +impl From<&PbHummockVersionDelta> for HummockVersionDelta { + fn from(pb_version_delta: &PbHummockVersionDelta) -> Self { + Self { + id: pb_version_delta.id, + prev_id: pb_version_delta.prev_id, + group_deltas: pb_version_delta + .group_deltas + .iter() + .map(|(group_id, deltas)| { + (*group_id as CompactionGroupId, GroupDeltas::from(deltas)) + }) + .collect(), + max_committed_epoch: pb_version_delta.max_committed_epoch, + safe_epoch: pb_version_delta.safe_epoch, + trivial_move: pb_version_delta.trivial_move, + new_table_watermarks: pb_version_delta + .new_table_watermarks + .iter() + .map(|(table_id, watermarks)| { + (TableId::new(*table_id), TableWatermarks::from(watermarks)) + }) + .collect(), + removed_table_ids: pb_version_delta + .removed_table_ids + .iter() + .map(|table_id| TableId::new(*table_id)) + .collect(), + change_log_delta: pb_version_delta + .change_log_delta + .iter() + .map(|(table_id, log_delta)| { + ( + TableId::new(*table_id), + ChangeLogDelta { + new_log: log_delta.new_log.clone().map(Into::into), + truncate_epoch: log_delta.truncate_epoch, + }, + ) + }) + .collect(), + + state_table_info_delta: pb_version_delta + .state_table_info_delta + .iter() + .map(|(table_id, delta)| (TableId::new(*table_id), delta.clone())) + .collect(), + } + } +} + +impl From<&HummockVersionDelta> for PbHummockVersionDelta { + fn from(version_delta: &HummockVersionDelta) -> Self { + Self { + id: version_delta.id, + prev_id: version_delta.prev_id, + group_deltas: version_delta + .group_deltas + .iter() + .map(|(group_id, deltas)| (*group_id as _, deltas.into())) + .collect(), + max_committed_epoch: version_delta.max_committed_epoch, + safe_epoch: version_delta.safe_epoch, + trivial_move: version_delta.trivial_move, + new_table_watermarks: version_delta + .new_table_watermarks + .iter() + .map(|(table_id, watermarks)| (table_id.table_id, watermarks.into())) + .collect(), + removed_table_ids: version_delta + .removed_table_ids + .iter() + .map(|table_id| table_id.table_id) + .collect(), + change_log_delta: version_delta + .change_log_delta + .iter() + .map(|(table_id, log_delta)| (table_id.table_id, log_delta.into())) + .collect(), + state_table_info_delta: version_delta + .state_table_info_delta + .iter() + .map(|(table_id, delta)| (table_id.table_id, delta.clone())) + .collect(), + } + } +} + +impl From for PbHummockVersionDelta { + fn from(version_delta: HummockVersionDelta) -> Self { + Self { + id: version_delta.id, + prev_id: version_delta.prev_id, + group_deltas: version_delta + .group_deltas + .into_iter() + .map(|(group_id, deltas)| (group_id as _, deltas.into())) + .collect(), + max_committed_epoch: version_delta.max_committed_epoch, + safe_epoch: version_delta.safe_epoch, + trivial_move: version_delta.trivial_move, + new_table_watermarks: version_delta + .new_table_watermarks + .into_iter() + .map(|(table_id, watermarks)| (table_id.table_id, watermarks.into())) + .collect(), + removed_table_ids: version_delta + .removed_table_ids + .into_iter() + .map(|table_id| table_id.table_id) + .collect(), + change_log_delta: version_delta + .change_log_delta + .into_iter() + .map(|(table_id, log_delta)| (table_id.table_id, log_delta.into())) + .collect(), + state_table_info_delta: version_delta + .state_table_info_delta + .into_iter() + .map(|(table_id, delta)| (table_id.table_id, delta.clone())) + .collect(), + } + } +} + +impl From for HummockVersionDelta { + fn from(pb_version_delta: PbHummockVersionDelta) -> Self { + Self { + id: pb_version_delta.id, + prev_id: pb_version_delta.prev_id, + group_deltas: pb_version_delta + .group_deltas + .into_iter() + .map(|(group_id, deltas)| (group_id as CompactionGroupId, deltas.into())) + .collect(), + max_committed_epoch: pb_version_delta.max_committed_epoch, + safe_epoch: pb_version_delta.safe_epoch, + trivial_move: pb_version_delta.trivial_move, + new_table_watermarks: pb_version_delta + .new_table_watermarks + .into_iter() + .map(|(table_id, watermarks)| (TableId::new(table_id), watermarks.into())) + .collect(), + removed_table_ids: pb_version_delta + .removed_table_ids + .into_iter() + .map(TableId::new) + .collect(), + change_log_delta: pb_version_delta + .change_log_delta + .iter() + .map(|(table_id, log_delta)| { + ( + TableId::new(*table_id), + ChangeLogDelta { + new_log: log_delta.new_log.clone().map(Into::into), + truncate_epoch: log_delta.truncate_epoch, + }, + ) + }) + .collect(), + state_table_info_delta: pb_version_delta + .state_table_info_delta + .iter() + .map(|(table_id, delta)| (TableId::new(*table_id), delta.clone())) + .collect(), + } + } +} + +#[derive(Debug, PartialEq, Clone)] +pub struct IntraLevelDelta { + pub level_idx: u32, + pub l0_sub_level_id: u64, + pub removed_table_ids: Vec, + pub inserted_table_infos: Vec, + pub vnode_partition_count: u32, +} + +impl IntraLevelDelta { + pub fn estimated_encode_len(&self) -> usize { + size_of::() + + size_of::() + + self.removed_table_ids.len() * size_of::() + + self + .inserted_table_infos + .iter() + .map(|sst| sst.estimated_encode_len()) + .sum::() + + size_of::() + } +} + +impl From for IntraLevelDelta { + fn from(pb_intra_level_delta: PbIntraLevelDelta) -> Self { + Self { + 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_table_infos + .into_iter() + .map(SstableInfo::from) + .collect_vec(), + vnode_partition_count: pb_intra_level_delta.vnode_partition_count, + } + } +} + +impl From for PbIntraLevelDelta { + fn from(intra_level_delta: IntraLevelDelta) -> Self { + Self { + level_idx: intra_level_delta.level_idx, + 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 + .into_iter() + .map(|sst| sst.into()) + .collect_vec(), + vnode_partition_count: intra_level_delta.vnode_partition_count, + } + } +} + +impl From<&IntraLevelDelta> for PbIntraLevelDelta { + fn from(intra_level_delta: &IntraLevelDelta) -> Self { + Self { + level_idx: intra_level_delta.level_idx, + 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 + .iter() + .map(|sst| sst.into()) + .collect_vec(), + vnode_partition_count: intra_level_delta.vnode_partition_count, + } + } +} + +impl From<&PbIntraLevelDelta> for IntraLevelDelta { + fn from(pb_intra_level_delta: &PbIntraLevelDelta) -> Self { + Self { + 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_table_infos + .iter() + .map(SstableInfo::from) + .collect_vec(), + vnode_partition_count: pb_intra_level_delta.vnode_partition_count, + } + } +} + +impl IntraLevelDelta { + pub fn new( + level_idx: u32, + l0_sub_level_id: u64, + removed_table_ids: Vec, + inserted_table_infos: Vec, + vnode_partition_count: u32, + ) -> Self { + Self { + level_idx, + l0_sub_level_id, + removed_table_ids, + inserted_table_infos, + vnode_partition_count, + } + } +} + +#[derive(Debug, PartialEq, Clone)] +pub enum GroupDelta { + IntraLevel(IntraLevelDelta), + GroupConstruct(PbGroupConstruct), + GroupDestroy(PbGroupDestroy), + GroupMetaChange(PbGroupMetaChange), + + #[allow(dead_code)] + GroupTableChange(PbGroupTableChange), +} + +impl From for GroupDelta { + fn from(pb_group_delta: PbGroupDelta) -> Self { + match pb_group_delta.delta_type { + Some(PbDeltaType::IntraLevel(pb_intra_level_delta)) => { + GroupDelta::IntraLevel(IntraLevelDelta::from(pb_intra_level_delta)) + } + Some(PbDeltaType::GroupConstruct(pb_group_construct)) => { + GroupDelta::GroupConstruct(pb_group_construct) + } + Some(PbDeltaType::GroupDestroy(pb_group_destroy)) => { + GroupDelta::GroupDestroy(pb_group_destroy) + } + Some(PbDeltaType::GroupMetaChange(pb_group_meta_change)) => { + GroupDelta::GroupMetaChange(pb_group_meta_change) + } + Some(PbDeltaType::GroupTableChange(pb_group_table_change)) => { + GroupDelta::GroupTableChange(pb_group_table_change) + } + None => panic!("delta_type is not set"), + } + } +} + +impl From for PbGroupDelta { + fn from(group_delta: GroupDelta) -> Self { + match group_delta { + GroupDelta::IntraLevel(intra_level_delta) => PbGroupDelta { + delta_type: Some(PbDeltaType::IntraLevel(intra_level_delta.into())), + }, + GroupDelta::GroupConstruct(pb_group_construct) => PbGroupDelta { + delta_type: Some(PbDeltaType::GroupConstruct(pb_group_construct)), + }, + GroupDelta::GroupDestroy(pb_group_destroy) => PbGroupDelta { + delta_type: Some(PbDeltaType::GroupDestroy(pb_group_destroy)), + }, + GroupDelta::GroupMetaChange(pb_group_meta_change) => PbGroupDelta { + delta_type: Some(PbDeltaType::GroupMetaChange(pb_group_meta_change)), + }, + GroupDelta::GroupTableChange(pb_group_table_change) => PbGroupDelta { + delta_type: Some(PbDeltaType::GroupTableChange(pb_group_table_change)), + }, + } + } +} + +impl From<&GroupDelta> for PbGroupDelta { + fn from(group_delta: &GroupDelta) -> Self { + match group_delta { + GroupDelta::IntraLevel(intra_level_delta) => PbGroupDelta { + delta_type: Some(PbDeltaType::IntraLevel(intra_level_delta.into())), + }, + GroupDelta::GroupConstruct(pb_group_construct) => PbGroupDelta { + delta_type: Some(PbDeltaType::GroupConstruct(pb_group_construct.clone())), + }, + GroupDelta::GroupDestroy(pb_group_destroy) => PbGroupDelta { + delta_type: Some(PbDeltaType::GroupDestroy(pb_group_destroy.clone())), + }, + GroupDelta::GroupMetaChange(pb_group_meta_change) => PbGroupDelta { + delta_type: Some(PbDeltaType::GroupMetaChange(pb_group_meta_change.clone())), + }, + GroupDelta::GroupTableChange(pb_group_table_change) => PbGroupDelta { + delta_type: Some(PbDeltaType::GroupTableChange(pb_group_table_change.clone())), + }, + } + } +} + +impl From<&PbGroupDelta> for GroupDelta { + fn from(pb_group_delta: &PbGroupDelta) -> Self { + match &pb_group_delta.delta_type { + Some(PbDeltaType::IntraLevel(pb_intra_level_delta)) => { + GroupDelta::IntraLevel(IntraLevelDelta::from(pb_intra_level_delta)) + } + Some(PbDeltaType::GroupConstruct(pb_group_construct)) => { + GroupDelta::GroupConstruct(pb_group_construct.clone()) + } + Some(PbDeltaType::GroupDestroy(pb_group_destroy)) => { + GroupDelta::GroupDestroy(pb_group_destroy.clone()) + } + Some(PbDeltaType::GroupMetaChange(pb_group_meta_change)) => { + GroupDelta::GroupMetaChange(pb_group_meta_change.clone()) + } + Some(PbDeltaType::GroupTableChange(pb_group_table_change)) => { + GroupDelta::GroupTableChange(pb_group_table_change.clone()) + } + None => panic!("delta_type is not set"), + } + } +} + +#[derive(Debug, PartialEq, Clone, Default)] +pub struct GroupDeltas { + pub group_deltas: Vec, +} + +impl From for GroupDeltas { + fn from(pb_group_deltas: PbGroupDeltas) -> Self { + Self { + group_deltas: pb_group_deltas + .group_deltas + .into_iter() + .map(GroupDelta::from) + .collect_vec(), + } + } +} + +impl From for PbGroupDeltas { + fn from(group_deltas: GroupDeltas) -> Self { + Self { + group_deltas: group_deltas + .group_deltas + .into_iter() + .map(|group_delta| group_delta.into()) + .collect_vec(), + } + } +} + +impl From<&GroupDeltas> for PbGroupDeltas { + fn from(group_deltas: &GroupDeltas) -> Self { + Self { + group_deltas: group_deltas + .group_deltas + .iter() + .map(|group_delta| group_delta.into()) + .collect_vec(), + } + } +} + +impl From<&PbGroupDeltas> for GroupDeltas { + fn from(pb_group_deltas: &PbGroupDeltas) -> Self { + Self { + group_deltas: pb_group_deltas + .group_deltas + .iter() + .map(GroupDelta::from) + .collect_vec(), + } + } +} + +impl GroupDeltas { + pub fn to_protobuf(&self) -> PbGroupDeltas { + self.into() + } +} diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 9f862e3300dc..78a49788b6c5 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -30,15 +30,18 @@ pub(crate) mod tests { use risingwave_common::util::epoch::{test_epoch, Epoch, EpochExt}; use risingwave_common_service::NotificationClient; use risingwave_hummock_sdk::can_concat; + use risingwave_hummock_sdk::compact_task::CompactTask; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{ next_key, prefix_slice_with_vnode, prefixed_range_with_vnode, FullKey, TableKey, TABLE_PREFIX_LEN, }; - use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; + use risingwave_hummock_sdk::key_range::KeyRange; + use risingwave_hummock_sdk::level::InputLevel; + use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; use risingwave_hummock_sdk::table_watermark::{ - ReadTableWatermark, VnodeWatermark, WatermarkDirection, + ReadTableWatermark, TableWatermarks, VnodeWatermark, WatermarkDirection, }; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_meta::hummock::compaction::compaction_config::CompactionConfigBuilder; @@ -51,10 +54,7 @@ pub(crate) mod tests { }; use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_pb::common::{HostAddress, WorkerType}; - use risingwave_pb::hummock::table_watermarks::PbEpochNewWatermarks; - use risingwave_pb::hummock::{ - CompactTask, InputLevel, KeyRange, SstableInfo, TableOption, TableWatermarks, - }; + use risingwave_pb::hummock::TableOption; use risingwave_pb::meta::add_worker_node_request::Property; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::filter_key_extractor::{ @@ -318,7 +318,7 @@ pub(crate) mod tests { .report_compact_task_for_test( result_task.task_id, Some(compact_task), - result_task.task_status(), + result_task.task_status, result_task.sorted_output_ssts, Some(to_prost_table_stats_map(task_stats)), ) @@ -473,7 +473,7 @@ pub(crate) mod tests { hummock_manager_ref .report_compact_task( result_task.task_id, - result_task.task_status(), + result_task.task_status, result_task.sorted_output_ssts, Some(to_prost_table_stats_map(task_stats)), ) @@ -807,7 +807,7 @@ pub(crate) mod tests { hummock_manager_ref .report_compact_task( result_task.task_id, - result_task.task_status(), + result_task.task_status, result_task.sorted_output_ssts, Some(to_prost_table_stats_map(task_stats)), ) @@ -1006,7 +1006,7 @@ pub(crate) mod tests { hummock_manager_ref .report_compact_task( result_task.task_id, - result_task.task_status(), + result_task.task_status, result_task.sorted_output_ssts, Some(to_prost_table_stats_map(task_stats)), ) @@ -1198,7 +1198,7 @@ pub(crate) mod tests { hummock_manager_ref .report_compact_task( result_task.task_id, - result_task.task_status(), + result_task.task_status, result_task.sorted_output_ssts, Some(to_prost_table_stats_map(task_stats)), ) @@ -1369,7 +1369,7 @@ pub(crate) mod tests { hummock_manager_ref .report_compact_task( result_task.task_id, - result_task.task_status(), + result_task.task_status, result_task.sorted_output_ssts, Some(to_prost_table_stats_map(task_stats)), ) @@ -1405,10 +1405,6 @@ pub(crate) mod tests { normal_tables.push(sstable_store.sstable(sst_info, &mut stats).await.unwrap()); } assert!(fast_ret.iter().all(|f| f.file_size < capacity * 6 / 5)); - println!( - "fast sstables file size: {:?}", - fast_ret.iter().map(|f| f.file_size).collect_vec(), - ); assert!(can_concat(&ret)); assert!(can_concat(&fast_ret)); let read_options = Arc::new(SstableIteratorReadOptions::default()); @@ -1538,7 +1534,6 @@ pub(crate) mod tests { sstable_store.clone(), ) .await; - println!("generate ssts size: {}", sst.file_size); ssts.push(sst); } let select_file_count = ssts.len() / 2; @@ -1547,12 +1542,12 @@ pub(crate) mod tests { input_ssts: vec![ InputLevel { level_idx: 5, - level_type: 1, + level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, table_infos: ssts.drain(..select_file_count).collect_vec(), }, InputLevel { level_idx: 6, - level_type: 1, + level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, table_infos: ssts, }, ], @@ -1769,12 +1764,12 @@ pub(crate) mod tests { input_ssts: vec![ InputLevel { level_idx: 5, - level_type: 1, + level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, table_infos: sst_infos.drain(..1).collect_vec(), }, InputLevel { level_idx: 6, - level_type: 1, + level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, table_infos: sst_infos, }, ], @@ -1885,10 +1880,6 @@ pub(crate) mod tests { max_sst_file_size = std::cmp::max(max_sst_file_size, sst_info.file_size); sst_infos.push(sst_info); } - println!( - "input data: {}", - sst_infos.iter().map(|sst| sst.file_size).sum::(), - ); let target_file_size = max_sst_file_size / 4; let mut table_watermarks = BTreeMap::default(); @@ -1907,13 +1898,11 @@ pub(crate) mod tests { table_watermarks.insert( 1, TableWatermarks { - epoch_watermarks: vec![PbEpochNewWatermarks { - watermarks: vec![ - VnodeWatermark::new(bitmap.clone(), watermark_key.clone()).to_protobuf() - ], - epoch: test_epoch(500), - }], - is_ascending: true, + watermarks: vec![( + test_epoch(500), + vec![VnodeWatermark::new(bitmap.clone(), watermark_key.clone())].into(), + )], + direction: WatermarkDirection::Ascending, }, ); @@ -1921,12 +1910,12 @@ pub(crate) mod tests { input_ssts: vec![ InputLevel { level_idx: 5, - level_type: 1, + level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, table_infos: sst_infos.drain(..1).collect_vec(), }, InputLevel { level_idx: 6, - level_type: 1, + level_type: risingwave_pb::hummock::LevelType::Nonoverlapping, table_infos: sst_infos, }, ], @@ -1943,12 +1932,7 @@ pub(crate) mod tests { ..Default::default() }; let (ret, fast_ret) = run_fast_and_normal_runner(compact_ctx.clone(), task).await; - println!( - "normal compact result data: {}, fast compact result data: {}", - ret.iter().map(|sst| sst.file_size).sum::(), - fast_ret.iter().map(|sst| sst.file_size).sum::(), - ); - // check_compaction_result(compact_ctx.sstable_store, ret.clone(), fast_ret, target_file_size).await; + let mut fast_tables = Vec::with_capacity(fast_ret.len()); let mut normal_tables = Vec::with_capacity(ret.len()); let mut stats = StoreLocalStatistic::default(); diff --git a/src/storage/hummock_test/src/hummock_read_version_tests.rs b/src/storage/hummock_test/src/hummock_read_version_tests.rs index 310f16ba9c6a..67c3aa8b059b 100644 --- a/src/storage/hummock_test/src/hummock_read_version_tests.rs +++ b/src/storage/hummock_test/src/hummock_read_version_tests.rs @@ -24,9 +24,10 @@ use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::{test_epoch, EpochExt}; use risingwave_hummock_sdk::key::{key_with_epoch, map_table_key_range}; +use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::LocalSstableInfo; use risingwave_meta::hummock::test_utils::setup_compute_env; -use risingwave_pb::hummock::{KeyRange, SstableInfo}; use risingwave_storage::hummock::event_handler::TEST_LOCAL_INSTANCE_ID; use risingwave_storage::hummock::iterator::test_utils::{ iterator_test_table_key_of, iterator_test_user_key_of, @@ -153,11 +154,13 @@ async fn test_read_version_basic() { LocalSstableInfo::for_test(SstableInfo { object_id: 1, sst_id: 1, - key_range: Some(KeyRange { - left: key_with_epoch(iterator_test_user_key_of(1).encode(), test_epoch(1)), - right: key_with_epoch(iterator_test_user_key_of(2).encode(), test_epoch(2)), + key_range: KeyRange { + left: key_with_epoch(iterator_test_user_key_of(1).encode(), test_epoch(1)) + .into(), + right: key_with_epoch(iterator_test_user_key_of(2).encode(), test_epoch(2)) + .into(), right_exclusive: false, - }), + }, file_size: 1, table_ids: vec![0], meta_offset: 1, @@ -169,11 +172,13 @@ async fn test_read_version_basic() { LocalSstableInfo::for_test(SstableInfo { object_id: 2, sst_id: 2, - key_range: Some(KeyRange { - left: key_with_epoch(iterator_test_user_key_of(3).encode(), test_epoch(3)), - right: key_with_epoch(iterator_test_user_key_of(3).encode(), test_epoch(3)), + key_range: KeyRange { + left: key_with_epoch(iterator_test_user_key_of(3).encode(), test_epoch(3)) + .into(), + right: key_with_epoch(iterator_test_user_key_of(3).encode(), test_epoch(3)) + .into(), right_exclusive: false, - }), + }, file_size: 1, table_ids: vec![0], meta_offset: 1, @@ -233,8 +238,8 @@ async fn test_read_version_basic() { let staging_ssts = staging_sst_iter.cloned().collect_vec(); assert_eq!(2, staging_ssts.len()); - assert_eq!(1, staging_ssts[0].get_object_id()); - assert_eq!(2, staging_ssts[1].get_object_id()); + assert_eq!(1, staging_ssts[0].object_id); + assert_eq!(2, staging_ssts[1].object_id); } { @@ -257,7 +262,7 @@ async fn test_read_version_basic() { let staging_ssts = staging_sst_iter.cloned().collect_vec(); assert_eq!(1, staging_ssts.len()); - assert_eq!(2, staging_ssts[0].get_object_id()); + assert_eq!(2, staging_ssts[0].object_id); } } diff --git a/src/storage/hummock_test/src/mock_notification_client.rs b/src/storage/hummock_test/src/mock_notification_client.rs index 906781a7725b..0de4805bdef7 100644 --- a/src/storage/hummock_test/src/mock_notification_client.rs +++ b/src/storage/hummock_test/src/mock_notification_client.rs @@ -62,7 +62,7 @@ impl NotificationClient for MockNotificationClient { let hummock_version = self.hummock_manager.get_current_version().await; let meta_snapshot = MetaSnapshot { - hummock_version: Some(hummock_version.to_protobuf()), + hummock_version: Some(hummock_version.into()), version: Some(Default::default()), meta_backup_manifest_id: Some(MetaBackupManifestId { id: 0 }), hummock_write_limits: Some(WriteLimits { diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 5c13d73f07ec..8df417072207 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -1302,7 +1302,7 @@ async fn test_gc_watermark_and_clear_shared_buffer() { sync_result .uncommitted_ssts .iter() - .map(|LocalSstableInfo { sst_info, .. }| sst_info.get_object_id()) + .map(|LocalSstableInfo { sst_info, .. }| sst_info.object_id) .min() .unwrap() }; diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index 3074bba6dd75..f5ee41783813 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -213,7 +213,7 @@ pub async fn compact_once( hummock_manager_ref .report_compact_task( result_task.task_id, - result_task.task_status(), + result_task.task_status, result_task.sorted_output_ssts, Some(to_prost_table_stats_map(task_stats)), ) @@ -417,16 +417,10 @@ async fn test_syncpoints_get_in_delete_range_boundary() { .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 - .as_ref() - .unwrap() - .right_exclusive - ); + assert!(base_level.table_infos[0].key_range.right_exclusive); assert_eq!( - user_key(&base_level.table_infos[0].key_range.as_ref().unwrap().right), - user_key(&base_level.table_infos[1].key_range.as_ref().unwrap().left), + user_key(&base_level.table_infos[0].key_range.right), + user_key(&base_level.table_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 63b59366195f..6eb49d627b34 100644 --- a/src/storage/src/hummock/compactor/compaction_utils.rs +++ b/src/storage/src/hummock/compactor/compaction_utils.rs @@ -18,18 +18,18 @@ use std::ops::Bound; use std::sync::atomic::{AtomicU64, Ordering}; use std::sync::Arc; +use bytes::Bytes; use itertools::Itertools; use risingwave_common::constants::hummock::CompactionFilterFlag; +use risingwave_hummock_sdk::compact_task::CompactTask; use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::key_range::KeyRange; -use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::TableStatsMap; use risingwave_hummock_sdk::{can_concat, EpochWithGap, KeyComparator}; -use risingwave_pb::hummock::compact_task::TaskType; -use risingwave_pb::hummock::{ - compact_task, BloomFilterType, CompactTask, LevelType, PbKeyRange, SstableInfo, TableSchema, -}; +use risingwave_pb::hummock::compact_task::PbTaskType; +use risingwave_pb::hummock::{BloomFilterType, PbLevelType, PbTableSchema}; use tokio::time::Instant; pub use super::context::CompactorContext; @@ -127,7 +127,7 @@ pub struct TaskConfig { /// change. For an divided SST as input, a dropped key shouldn't be counted if its table id /// doesn't belong to this divided SST. See `Compactor::compact_and_build_sst`. pub stats_target_table_ids: Option>, - pub task_type: compact_task::TaskType, + pub task_type: PbTaskType, pub is_target_l0_or_lbase: bool, pub use_block_based_filter: bool, @@ -135,7 +135,7 @@ pub struct TaskConfig { /// `TableId` -> `TableSchema` /// Schemas in `table_schemas` are at least as new as the one used to create `input_ssts`. /// For a table with schema existing in `table_schemas`, its columns not in `table_schemas` but in `input_ssts` can be safely dropped. - pub table_schemas: HashMap, + pub table_schemas: HashMap, /// `disable_drop_column_optimization` should only be set in benchmark. pub disable_drop_column_optimization: bool, } @@ -178,7 +178,7 @@ fn generate_splits_fast( compaction_size: u64, context: &CompactorContext, max_sub_compaction: u32, -) -> Vec { +) -> Vec { let worker_num = context.compaction_executor.worker_num(); let parallel_compact_size = (context.storage_opts.parallel_compact_size_mb as u64) << 20; @@ -190,7 +190,7 @@ fn generate_splits_fast( ); let mut indexes = vec![]; for sst in sstable_infos { - let key_range = sst.key_range.as_ref().unwrap(); + let key_range = &sst.key_range; indexes.push( FullKey { user_key: FullKey::decode(&key_range.left).user_key, @@ -213,13 +213,13 @@ fn generate_splits_fast( } let mut splits = vec![]; - splits.push(PbKeyRange::new(vec![], vec![])); + splits.push(KeyRange::default()); let parallel_key_count = indexes.len() / parallelism; let mut last_split_key_count = 0; for key in indexes { if last_split_key_count >= parallel_key_count { - splits.last_mut().unwrap().right.clone_from(&key); - splits.push(PbKeyRange::new(key.clone(), vec![])); + splits.last_mut().unwrap().right = Bytes::from(key.clone()); + splits.push(KeyRange::new(Bytes::from(key.clone()), Bytes::default())); last_split_key_count = 0; } last_split_key_count += 1; @@ -233,7 +233,7 @@ pub async fn generate_splits( compaction_size: u64, context: &CompactorContext, max_sub_compaction: u32, -) -> HummockResult> { +) -> HummockResult> { const MAX_FILE_COUNT: usize = 32; let parallel_compact_size = (context.storage_opts.parallel_compact_size_mb as u64) << 20; if compaction_size > parallel_compact_size { @@ -271,7 +271,7 @@ pub async fn generate_splits( // sort by key, as for every data block has the same size; indexes.sort_by(|a, b| KeyComparator::compare_encoded_full_key(a.1.as_ref(), b.1.as_ref())); let mut splits = vec![]; - splits.push(PbKeyRange::new(vec![], vec![])); + splits.push(KeyRange::default()); let parallelism = calculate_task_parallelism_impl( context.compaction_executor.worker_num(), @@ -292,8 +292,8 @@ pub async fn generate_splits( && !last_key.eq(&key) && remaining_size > parallel_compact_size { - splits.last_mut().unwrap().right.clone_from(&key); - splits.push(PbKeyRange::new(key.clone(), vec![])); + splits.last_mut().unwrap().right = Bytes::from(key.clone()); + splits.push(KeyRange::new(Bytes::from(key.clone()), Bytes::default())); last_buffer_size = data_size; } else { last_buffer_size += data_size; @@ -357,7 +357,7 @@ pub async fn check_compaction_result( } // Do not need to filter the table because manager has done it. - if level.level_type == LevelType::Nonoverlapping as i32 { + if level.level_type == PbLevelType::Nonoverlapping { debug_assert!(can_concat(&level.table_infos)); del_iter.add_concat_iter(level.table_infos.clone(), context.sstable_store.clone()); @@ -513,7 +513,7 @@ pub fn optimize_by_copy_block(compact_task: &CompactTask, context: &CompactorCon let all_ssts_are_blocked_filter = sstable_infos .iter() - .all(|table_info| table_info.bloom_filter_kind() == BloomFilterType::Blocked); + .all(|table_info| table_info.bloom_filter_kind == BloomFilterType::Blocked); let delete_key_count = sstable_infos .iter() @@ -553,7 +553,7 @@ pub fn optimize_by_copy_block(compact_task: &CompactTask, context: &CompactorCon && compaction_size < context.storage_opts.compactor_fast_max_compact_task_size && delete_key_count * 100 < context.storage_opts.compactor_fast_max_compact_delete_ratio as u64 * total_key_count - && compact_task.task_type() == TaskType::Dynamic + && compact_task.task_type == PbTaskType::Dynamic } pub async fn generate_splits_for_task( @@ -583,7 +583,7 @@ pub async fn generate_splits_for_task( &sstable_infos, compaction_size, context, - compact_task.get_max_sub_compaction(), + compact_task.max_sub_compaction, ) .await?; if !splits.is_empty() { @@ -667,7 +667,7 @@ pub fn calculate_task_parallelism(compact_task: &CompactTask, context: &Compacto context.compaction_executor.worker_num(), parallel_compact_size, compaction_size, - compact_task.get_max_sub_compaction(), + compact_task.max_sub_compaction, ) } diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 0336f6f542fa..773b2d565550 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -23,14 +23,16 @@ use risingwave_common::util::value_encoding::column_aware_row_encoding::try_drop use risingwave_hummock_sdk::compact::{ compact_task_to_string, estimate_memory_for_compact_task, statistics_compact_task, }; +use risingwave_hummock_sdk::compact_task::CompactTask; use risingwave_hummock_sdk::key::{FullKey, FullKeyTracker}; use risingwave_hummock_sdk::key_range::{KeyRange, KeyRangeCommon}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::{add_table_stats_map, TableStats, TableStatsMap}; use risingwave_hummock_sdk::{ can_concat, compact_task_output_to_string, HummockSstableObjectId, KeyComparator, }; use risingwave_pb::hummock::compact_task::TaskStatus; -use risingwave_pb::hummock::{CompactTask, LevelType, SstableInfo}; +use risingwave_pb::hummock::LevelType; use thiserror_ext::AsReport; use tokio::sync::oneshot::Receiver; @@ -94,8 +96,8 @@ impl CompactorRunner { BlockedXor16FilterBuilder::is_kv_count_too_large(kv_count) || task.target_level > 0; let key_range = KeyRange { - left: Bytes::copy_from_slice(task.splits[split_index].get_left()), - right: Bytes::copy_from_slice(task.splits[split_index].get_right()), + left: task.splits[split_index].left.clone(), + right: task.splits[split_index].right.clone(), right_exclusive: true, }; @@ -108,7 +110,7 @@ impl CompactorRunner { gc_delete_keys: task.gc_delete_keys, watermark: task.watermark, stats_target_table_ids: Some(HashSet::from_iter(task.existing_table_ids.clone())), - task_type: task.task_type(), + task_type: task.task_type, is_target_l0_or_lbase: task.target_level == 0 || task.target_level == task.base_level, use_block_based_filter, @@ -173,18 +175,17 @@ impl CompactorRunner { .table_infos .iter() .filter(|table_info| { - let key_range = KeyRange::from(table_info.key_range.as_ref().unwrap()); let table_ids = &table_info.table_ids; let exist_table = table_ids .iter() .any(|table_id| self.compact_task.existing_table_ids.contains(table_id)); - self.key_range.full_key_overlap(&key_range) && exist_table + self.key_range.full_key_overlap(&table_info.key_range) && exist_table }) .cloned() .collect_vec(); // Do not need to filter the table because manager has done it. - if level.level_type == LevelType::Nonoverlapping as i32 { + if level.level_type == LevelType::Nonoverlapping { debug_assert!(can_concat(&level.table_infos)); table_iters.push(ConcatSstableIterator::new( self.compact_task.existing_table_ids.clone(), @@ -202,7 +203,7 @@ impl CompactorRunner { sst_groups.len() ); for table_infos in sst_groups { - assert!(can_concat(&table_infos)); + assert!(can_concat(table_infos.as_slice())); table_iters.push(ConcatSstableIterator::new( self.compact_task.existing_table_ids.clone(), table_infos, @@ -243,7 +244,7 @@ pub fn partition_overlapping_sstable_infos( ) -> Vec> { pub struct SstableGroup { ssts: Vec, - max_right_bound: Vec, + max_right_bound: Bytes, } impl PartialEq for SstableGroup { @@ -265,8 +266,8 @@ pub fn partition_overlapping_sstable_infos( } let mut groups: BinaryHeap = BinaryHeap::default(); origin_infos.sort_by(|a, b| { - let x = a.key_range.as_ref().unwrap(); - let y = b.key_range.as_ref().unwrap(); + let x = &a.key_range; + let y = &b.key_range; KeyComparator::compare_encoded_full_key(&x.left, &y.left) }); for sst in origin_infos { @@ -274,17 +275,15 @@ pub fn partition_overlapping_sstable_infos( if let Some(mut prev_group) = groups.peek_mut() { if KeyComparator::encoded_full_key_less_than( &prev_group.max_right_bound, - &sst.key_range.as_ref().unwrap().left, + &sst.key_range.left, ) { - prev_group - .max_right_bound - .clone_from(&sst.key_range.as_ref().unwrap().right); + prev_group.max_right_bound.clone_from(&sst.key_range.right); prev_group.ssts.push(sst); continue; } } groups.push(SstableGroup { - max_right_bound: sst.key_range.as_ref().unwrap().right.clone(), + max_right_bound: sst.key_range.right.clone(), ssts: vec![sst], }); } @@ -577,7 +576,7 @@ pub(crate) fn compact_done( task_status: TaskStatus, ) -> (CompactTask, HashMap) { let mut table_stats_map = TableStatsMap::default(); - compact_task.set_task_status(task_status); + compact_task.task_status = task_status; compact_task .sorted_output_ssts .reserve(compact_task.splits.len()); diff --git a/src/storage/src/hummock/compactor/fast_compactor_runner.rs b/src/storage/src/hummock/compactor/fast_compactor_runner.rs index 81f0210d945c..227a50cb5813 100644 --- a/src/storage/src/hummock/compactor/fast_compactor_runner.rs +++ b/src/storage/src/hummock/compactor/fast_compactor_runner.rs @@ -22,11 +22,12 @@ use std::time::Instant; use await_tree::InstrumentAwait; use bytes::Bytes; use itertools::Itertools; +use risingwave_hummock_sdk::compact_task::CompactTask; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::TableStats; use risingwave_hummock_sdk::{can_concat, compact_task_to_string, EpochWithGap, LocalSstableInfo}; -use risingwave_pb::hummock::{CompactTask, SstableInfo}; use crate::filter_key_extractor::FilterKeyExtractorImpl; use crate::hummock::block_stream::BlockDataStream; @@ -292,7 +293,7 @@ impl CompactorRunner { gc_delete_keys: task.gc_delete_keys, watermark: task.watermark, stats_target_table_ids: Some(HashSet::from_iter(task.existing_table_ids.clone())), - task_type: task.task_type(), + task_type: task.task_type, is_target_l0_or_lbase: task.target_level == 0 || task.target_level == task.base_level, table_vnode_partition: task.table_vnode_partition.clone(), use_block_based_filter: true, diff --git a/src/storage/src/hummock/compactor/iterator.rs b/src/storage/src/hummock/compactor/iterator.rs index 5431c51270ef..e96164cb6be0 100644 --- a/src/storage/src/hummock/compactor/iterator.rs +++ b/src/storage/src/hummock/compactor/iterator.rs @@ -23,8 +23,8 @@ use fail::fail_point; use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::KeyComparator; -use risingwave_pb::hummock::SstableInfo; use crate::hummock::block_stream::BlockDataStream; use crate::hummock::compactor::task_progress::TaskProgress; @@ -242,8 +242,8 @@ impl SstableStreamIterator { fn sst_debug_info(&self) -> String { format!( "object_id={}, sst_id={}, meta_offset={}, table_ids={:?}", - self.sstable_info.get_object_id(), - self.sstable_info.get_sst_id(), + self.sstable_info.object_id, + self.sstable_info.sst_id, self.sstable_info.meta_offset, self.sstable_info.table_ids ) @@ -478,7 +478,7 @@ impl HummockIterator for ConcatSstableIterator { // Note that we need to use `<` instead of `<=` to ensure that all keys in an SST // (including its max. key) produce the same search result. - let max_sst_key = &table.key_range.as_ref().unwrap().right; + let max_sst_key = &table.key_range.right; FullKey::decode(max_sst_key).cmp(&seek_key) == Ordering::Less }); diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index 898d91d530b2..12e009bb60cb 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -15,12 +15,13 @@ mod compaction_executor; mod compaction_filter; pub mod compaction_utils; +use risingwave_hummock_sdk::compact_task::{CompactTask, ValidationTask}; use risingwave_pb::compactor::{dispatch_compaction_task_request, DispatchCompactionTaskRequest}; use risingwave_pb::hummock::report_compaction_task_request::{ Event as ReportCompactionTaskEvent, HeartBeat as SharedHeartBeat, ReportTask as ReportSharedTask, }; -use risingwave_pb::hummock::{CompactTask, ReportFullScanTaskRequest, ReportVacuumTaskRequest}; +use risingwave_pb::hummock::{PbCompactTask, ReportFullScanTaskRequest, ReportVacuumTaskRequest}; use risingwave_rpc_client::GrpcCompactorProxyClient; use thiserror_ext::AsReport; use tokio::sync::mpsc; @@ -188,7 +189,7 @@ impl Compactor { debug_assert!(split_table_outputs .iter() - .all(|table_info| table_info.sst_info.get_table_ids().is_sorted())); + .all(|table_info| table_info.sst_info.table_ids.is_sorted())); if task_id.is_some() { // skip shared buffer compaction @@ -427,8 +428,12 @@ pub fn start_compactor( if let Err(e) = request_sender.send(SubscribeCompactionEventRequest { event: Some(RequestEvent::ReportTask(ReportTask { task_id: compact_task.task_id, - task_status: compact_task.task_status, - sorted_output_ssts: compact_task.sorted_output_ssts.clone(), + task_status: compact_task.task_status.into(), + sorted_output_ssts: compact_task + .sorted_output_ssts + .iter() + .map(|sst| sst.into()) + .collect(), table_stats_change: to_prost_table_stats_map(table_stats), })), create_at: SystemTime::now() @@ -465,6 +470,7 @@ pub fn start_compactor( match event { ResponseEvent::CompactTask(compact_task) => { + let compact_task = CompactTask::from(compact_task); let parallelism = calculate_task_parallelism(&compact_task, &context); @@ -531,9 +537,7 @@ pub fn start_compactor( Err(err) => { tracing::warn!(error = %err.as_report(), "Failed to track pending SST object id"); let mut compact_task = compact_task; - compact_task.set_task_status( - TaskStatus::TrackSstObjectIdFailed, - ); + compact_task.task_status = TaskStatus::TrackSstObjectIdFailed; ((compact_task, HashMap::default()), None) } }; @@ -548,7 +552,7 @@ pub fn start_compactor( let enable_check_compaction_result = context.storage_opts.check_compaction_result; - let need_check_task = !compact_task.sorted_output_ssts.is_empty() && compact_task.task_status() == TaskStatus::Success; + let need_check_task = !compact_task.sorted_output_ssts.is_empty() && compact_task.task_status == TaskStatus::Success; if enable_check_compaction_result && need_check_task { match check_compaction_result(&compact_task, context.clone()) @@ -606,6 +610,7 @@ pub fn start_compactor( }); } ResponseEvent::ValidationTask(validation_task) => { + let validation_task = ValidationTask::from(validation_task); executor.spawn(async move { validate_ssts(validation_task, context.sstable_store.clone()) .await; @@ -729,6 +734,7 @@ pub fn start_shared_compactor( SharedComapctorObjectIdManager::new(output_object_ids_deque, cloned_grpc_proxy_client.clone(), context.storage_opts.sstable_id_remote_fetch_number); match dispatch_task.unwrap() { dispatch_compaction_task_request::Task::CompactTask(compact_task) => { + let compact_task = CompactTask::from(&compact_task); let (tx, rx) = tokio::sync::oneshot::channel(); let task_id = compact_task.task_id; shutdown.lock().unwrap().insert(task_id, tx); @@ -744,7 +750,7 @@ pub fn start_shared_compactor( shutdown.lock().unwrap().remove(&task_id); let report_compaction_task_request = ReportCompactionTaskRequest { event: Some(ReportCompactionTaskEvent::ReportTask(ReportSharedTask { - compact_task: Some(compact_task.clone()), + compact_task: Some(PbCompactTask::from(&compact_task)), table_stats_change: to_prost_table_stats_map(table_stats), })), }; @@ -756,11 +762,11 @@ pub fn start_shared_compactor( Ok(_) => { // TODO: remove this method after we have running risingwave cluster with fast compact algorithm stably for a long time. let enable_check_compaction_result = context.storage_opts.check_compaction_result; - let need_check_task = !compact_task.sorted_output_ssts.is_empty() && compact_task.task_status() == TaskStatus::Success; + let need_check_task = !compact_task.sorted_output_ssts.is_empty() && compact_task.task_status == TaskStatus::Success; if enable_check_compaction_result && need_check_task { match check_compaction_result(&compact_task, context.clone()).await { Err(e) => { - tracing::warn!(error = %e.as_report(), "Failed to check compaction task {}", compact_task.task_id); + tracing::warn!(error = %e.as_report(), "Failed to check compaction task {}", task_id); }, Ok(true) => (), Ok(false) => { @@ -818,6 +824,7 @@ pub fn start_shared_compactor( } } dispatch_compaction_task_request::Task::ValidationTask(validation_task) => { + let validation_task = ValidationTask::from(validation_task); validate_ssts(validation_task, context.sstable_store.clone()).await; } dispatch_compaction_task_request::Task::CancelCompactTask(cancel_compact_task) => { diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index addb2d08d5fc..cd177e4cf34f 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -53,7 +53,6 @@ use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::store::version::{ HummockReadVersion, StagingData, StagingSstableInfo, VersionUpdate, }; -use crate::hummock::utils::validate_table_key_range; use crate::hummock::{ HummockResult, MemoryLimiter, SstableObjectIdManager, SstableStoreRef, TrackerId, }; @@ -612,8 +611,6 @@ impl HummockEventHandler { HummockVersionUpdate::PinnedVersion(version) => *version, }; - validate_table_key_range(&newly_pinned_version); - pinned_version.new_pin_version(newly_pinned_version) } diff --git a/src/storage/src/hummock/event_handler/uploader/test_utils.rs b/src/storage/src/hummock/event_handler/uploader/test_utils.rs index 2fa574c72fc2..f866ef18f9e2 100644 --- a/src/storage/src/hummock/event_handler/uploader/test_utils.rs +++ b/src/storage/src/hummock/event_handler/uploader/test_utils.rs @@ -31,9 +31,11 @@ use risingwave_common::must_match; use risingwave_common::util::epoch::{test_epoch, EpochExt}; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{FullKey, TableKey}; +use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; -use risingwave_pb::hummock::{KeyRange, SstableInfo, StateTableInfoDelta}; +use risingwave_pb::hummock::StateTableInfoDelta; use spin::Mutex; use tokio::spawn; use tokio::sync::mpsc::unbounded_channel; @@ -159,11 +161,11 @@ pub(super) fn gen_sstable_info( vec![LocalSstableInfo::for_test(SstableInfo { object_id: gen_sst_object_id, sst_id: gen_sst_object_id, - key_range: Some(KeyRange { - left: start_full_key.encode(), - right: end_full_key.encode(), + key_range: KeyRange { + left: start_full_key.encode().into(), + right: end_full_key.encode().into(), right_exclusive: true, - }), + }, table_ids: vec![TEST_TABLE_ID.table_id], ..Default::default() })] diff --git a/src/storage/src/hummock/iterator/concat_delete_range_iterator.rs b/src/storage/src/hummock/iterator/concat_delete_range_iterator.rs index 35a9e53079f9..a7c5215439bf 100644 --- a/src/storage/src/hummock/iterator/concat_delete_range_iterator.rs +++ b/src/storage/src/hummock/iterator/concat_delete_range_iterator.rs @@ -15,8 +15,8 @@ use std::future::Future; use risingwave_hummock_sdk::key::{FullKey, PointRange, UserKey}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::HummockEpoch; -use risingwave_pb::hummock::SstableInfo; use crate::hummock::iterator::DeleteRangeIterator; use crate::hummock::sstable_store::SstableStoreRef; @@ -51,32 +51,26 @@ impl ConcatDeleteRangeIterator { && iter .next_extended_user_key() .left_user_key - .eq(&FullKey::decode( - &self.sstables[self.idx].key_range.as_ref().unwrap().right, - ) - .user_key) + .eq(&FullKey::decode(&self.sstables[self.idx].key_range.right).user_key) { // When the last range of the current sstable is equal to the first range of the // next sstable, the `next` method would return two same `PointRange`. So we // must skip one. let exclusive_range_start = iter.next_extended_user_key().is_exclude_left_key; - let last_key_in_sst_start = - iter.next_extended_user_key() - .left_user_key - .eq(&FullKey::decode( - &self.sstables[self.idx + 1].key_range.as_ref().unwrap().left, - ) - .user_key); + let last_key_in_sst_start = iter + .next_extended_user_key() + .left_user_key + .eq(&FullKey::decode(&self.sstables[self.idx + 1].key_range.left).user_key); iter.next().await?; if !iter.is_valid() && last_key_in_sst_start { self.seek_idx(self.idx + 1, None).await?; let next_range = self.next_extended_user_key(); debug_assert!(self.is_valid()); if next_range.is_exclude_left_key == exclusive_range_start - && next_range.left_user_key.eq(&FullKey::decode( - &self.sstables[self.idx].key_range.as_ref().unwrap().left, - ) - .user_key) + && next_range + .left_user_key + .eq(&FullKey::decode(&self.sstables[self.idx].key_range.left) + .user_key) { self.current.as_mut().unwrap().next().await?; } @@ -158,7 +152,7 @@ impl DeleteRangeIterator for ConcatDeleteRangeIterator { let mut idx = self .sstables .partition_point(|sst| { - FullKey::decode(&sst.key_range.as_ref().unwrap().left) + FullKey::decode(&sst.key_range.left) .user_key .le(&target_user_key) }) diff --git a/src/storage/src/hummock/iterator/concat_inner.rs b/src/storage/src/hummock/iterator/concat_inner.rs index 357d12e32036..a71a9d940186 100644 --- a/src/storage/src/hummock/iterator/concat_inner.rs +++ b/src/storage/src/hummock/iterator/concat_inner.rs @@ -16,7 +16,7 @@ use std::cmp::Ordering::{Equal, Greater, Less}; use std::sync::Arc; use risingwave_hummock_sdk::key::FullKey; -use risingwave_pb::hummock::SstableInfo; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use crate::hummock::iterator::{ DirectionEnum, HummockIterator, HummockIteratorDirection, ValueMeta, @@ -27,11 +27,11 @@ use crate::hummock::{HummockResult, SstableIteratorType, SstableStoreRef}; use crate::monitor::StoreLocalStatistic; fn smallest_key(sstable_info: &SstableInfo) -> &[u8] { - &sstable_info.key_range.as_ref().unwrap().left + &sstable_info.key_range.left } fn largest_key(sstable_info: &SstableInfo) -> &[u8] { - &sstable_info.key_range.as_ref().unwrap().right + &sstable_info.key_range.right } /// Served as the concrete implementation of `ConcatIterator` and `BackwardConcatIterator`. @@ -160,8 +160,7 @@ impl HummockIterator for ConcatIteratorInner { } DirectionEnum::Backward => { let ord = FullKey::decode(largest_key(table)).cmp(&key); - ord == Greater - || (ord == Equal && !table.key_range.as_ref().unwrap().right_exclusive) + ord == Greater || (ord == Equal && !table.key_range.right_exclusive) } }) .saturating_sub(1); // considering the boundary of 0 diff --git a/src/storage/src/hummock/iterator/delete_range_iterator.rs b/src/storage/src/hummock/iterator/delete_range_iterator.rs index f727990b118f..bcc2f3e3ea26 100644 --- a/src/storage/src/hummock/iterator/delete_range_iterator.rs +++ b/src/storage/src/hummock/iterator/delete_range_iterator.rs @@ -17,8 +17,8 @@ use std::future::Future; use risingwave_common::util::epoch::is_max_epoch; use risingwave_hummock_sdk::key::{PointRange, UserKey}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::HummockEpoch; -use risingwave_pb::hummock::SstableInfo; use crate::hummock::iterator::concat_delete_range_iterator::ConcatDeleteRangeIterator; use crate::hummock::shared_buffer::shared_buffer_batch::SharedBufferDeleteRangeIterator; diff --git a/src/storage/src/hummock/iterator/mod.rs b/src/storage/src/hummock/iterator/mod.rs index 8a14efc80106..fdfcd26a3a59 100644 --- a/src/storage/src/hummock/iterator/mod.rs +++ b/src/storage/src/hummock/iterator/mod.rs @@ -18,6 +18,7 @@ use std::ops::{Deref, DerefMut}; use std::sync::Arc; use more_asserts::{assert_gt, assert_lt}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use super::{ HummockResult, HummockValue, SstableIteratorReadOptions, SstableIteratorType, SstableStoreRef, @@ -55,7 +56,6 @@ pub use delete_range_iterator::{ DeleteRangeIterator, ForwardMergeRangeIterator, RangeIteratorTyped, }; use risingwave_common::catalog::TableId; -use risingwave_pb::hummock::SstableInfo; pub use skip_watermark::*; use crate::hummock::shared_buffer::shared_buffer_batch::SharedBufferBatch; diff --git a/src/storage/src/hummock/iterator/skip_watermark.rs b/src/storage/src/hummock/iterator/skip_watermark.rs index d335844eb400..11ee62dc462f 100644 --- a/src/storage/src/hummock/iterator/skip_watermark.rs +++ b/src/storage/src/hummock/iterator/skip_watermark.rs @@ -21,8 +21,9 @@ use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::safe_epoch_read_table_watermarks_impl; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::table_stats::{add_table_stats_map, TableStats, TableStatsMap}; -use risingwave_hummock_sdk::table_watermark::{ReadTableWatermark, WatermarkDirection}; -use risingwave_pb::hummock::PbTableWatermarks; +use risingwave_hummock_sdk::table_watermark::{ + ReadTableWatermark, TableWatermarks, WatermarkDirection, +}; use crate::hummock::iterator::{Forward, HummockIterator, ValueMeta}; use crate::hummock::value::HummockValue; @@ -53,7 +54,7 @@ impl> SkipWatermarkIterator { pub fn from_safe_epoch_watermarks( inner: I, - safe_epoch_watermarks: &BTreeMap, + safe_epoch_watermarks: &BTreeMap, ) -> Self { Self { inner, @@ -185,7 +186,7 @@ impl SkipWatermarkState { } pub fn from_safe_epoch_watermarks( - safe_epoch_watermarks: &BTreeMap, + safe_epoch_watermarks: &BTreeMap, ) -> Self { let watermarks = safe_epoch_read_table_watermarks_impl(safe_epoch_watermarks); Self::new(watermarks) diff --git a/src/storage/src/hummock/iterator/test_utils.rs b/src/storage/src/hummock/iterator/test_utils.rs index 9cc6d8dc7fcd..971e3b7f97c7 100644 --- a/src/storage/src/hummock/iterator/test_utils.rs +++ b/src/storage/src/hummock/iterator/test_utils.rs @@ -22,11 +22,11 @@ use risingwave_common::config::{MetricLevel, ObjectStoreConfig}; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::key::{prefix_slice_with_vnode, FullKey, TableKey, UserKey}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::{EpochWithGap, HummockEpoch, HummockSstableObjectId}; use risingwave_object_store::object::{ InMemObjectStore, ObjectStore, ObjectStoreImpl, ObjectStoreRef, }; -use risingwave_pb::hummock::SstableInfo; use crate::hummock::shared_buffer::shared_buffer_batch::SharedBufferValue; use crate::hummock::sstable::SstableIteratorReadOptions; diff --git a/src/storage/src/hummock/local_version/pinned_version.rs b/src/storage/src/hummock/local_version/pinned_version.rs index 5e89a135d825..8819d0162a5e 100644 --- a/src/storage/src/hummock/local_version/pinned_version.rs +++ b/src/storage/src/hummock/local_version/pinned_version.rs @@ -19,10 +19,9 @@ use std::time::{Duration, Instant}; use auto_enums::auto_enum; use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::level::{Level, Levels}; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{CompactionGroupId, HummockVersionId, INVALID_VERSION_ID}; -use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::PbLevel; use risingwave_rpc_client::HummockMetaClient; use thiserror_ext::AsReport; use tokio::sync::mpsc::error::TryRecvError; @@ -133,7 +132,7 @@ impl PinnedVersion { self.version.levels.get(&compaction_group_id).unwrap() } - pub fn levels(&self, table_id: TableId) -> impl Iterator { + pub fn levels(&self, table_id: TableId) -> impl Iterator { #[auto_enum(Iterator)] match self.version.state_table_info.info().get(&table_id) { Some(info) => { diff --git a/src/storage/src/hummock/mod.rs b/src/storage/src/hummock/mod.rs index 3974af300621..14ac9532c8cb 100644 --- a/src/storage/src/hummock/mod.rs +++ b/src/storage/src/hummock/mod.rs @@ -19,8 +19,8 @@ use std::sync::Arc; use bytes::Bytes; use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKeyRangeRef}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::{HummockEpoch, *}; -use risingwave_pb::hummock::SstableInfo; pub mod block_cache; pub use block_cache::*; diff --git a/src/storage/src/hummock/sstable/builder.rs b/src/storage/src/hummock/sstable/builder.rs index da904b26e82c..77399eb3e3d0 100644 --- a/src/storage/src/hummock/sstable/builder.rs +++ b/src/storage/src/hummock/sstable/builder.rs @@ -19,9 +19,11 @@ use std::sync::Arc; use bytes::{Bytes, BytesMut}; use risingwave_common::util::epoch::is_max_epoch; use risingwave_hummock_sdk::key::{user_key, FullKey, MAX_KEY_LEN}; +use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::{TableStats, TableStatsMap}; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; -use risingwave_pb::hummock::{BloomFilterType, SstableInfo}; +use risingwave_pb::hummock::BloomFilterType; use super::utils::CompressionAlgorithm; use super::{ @@ -498,12 +500,12 @@ impl SstableBuilder { let sst_info = SstableInfo { object_id: self.sstable_id, sst_id: self.sstable_id, - bloom_filter_kind: bloom_filter_kind as i32, - key_range: Some(risingwave_pb::hummock::KeyRange { - left: meta.smallest_key.clone(), - right: meta.largest_key.clone(), + bloom_filter_kind, + key_range: KeyRange { + left: Bytes::from(meta.smallest_key.clone()), + right: Bytes::from(meta.largest_key.clone()), right_exclusive, - }), + }, file_size: meta.estimated_size as u64, table_ids: self.table_ids.into_iter().collect(), meta_offset: meta.meta_offset, @@ -525,7 +527,7 @@ impl SstableBuilder { self.epoch_set.len() ); let bloom_filter_size = meta.bloom_filter.len(); - let sstable_file_size = sst_info.get_file_size() as usize; + let sstable_file_size = sst_info.file_size as usize; let writer_output = self.writer.finish(meta).await?; Ok(SstableBuilderOutput:: { @@ -715,13 +717,10 @@ pub(super) mod tests { let output = b.finish().await.unwrap(); let info = output.sst_info.sst_info; - assert_bytes_eq!( - test_key_of(0).encode(), - info.key_range.as_ref().unwrap().left - ); + assert_bytes_eq!(test_key_of(0).encode(), info.key_range.left); assert_bytes_eq!( test_key_of(TEST_KEYS_COUNT - 1).encode(), - info.key_range.as_ref().unwrap().right + info.key_range.right ); let (data, meta) = output.writer_output; assert_eq!(info.file_size, meta.estimated_size as u64); diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index 8843ddff7719..cea2c42529ce 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -26,12 +26,12 @@ use foyer::{ use futures::{future, StreamExt}; use itertools::Itertools; use risingwave_common::config::StorageMemoryConfig; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::{HummockSstableObjectId, OBJECT_SUFFIX}; use risingwave_hummock_trace::TracedCachePolicy; use risingwave_object_store::object::{ ObjectError, ObjectMetadataIter, ObjectStoreRef, ObjectStreamingUploader, }; -use risingwave_pb::hummock::SstableInfo; use serde::{Deserialize, Serialize}; use thiserror_ext::AsReport; use tokio::task::JoinHandle; @@ -572,7 +572,7 @@ impl SstableStore { sst: &SstableInfo, stats: &mut StoreLocalStatistic, ) -> impl Future> + Send + 'static { - let object_id = sst.get_object_id(); + let object_id = sst.object_id; let entry = self.meta_cache.fetch(object_id, || { let store = self.store.clone(); @@ -1131,8 +1131,8 @@ mod tests { use std::ops::Range; use std::sync::Arc; + use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::HummockSstableObjectId; - use risingwave_pb::hummock::SstableInfo; use super::{SstableStoreRef, SstableWriterOptions}; use crate::hummock::iterator::test_utils::{iterator_test_key_of, mock_sstable_store}; diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 511d9dd33814..f98ee38458b6 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -29,10 +29,10 @@ use risingwave_common_service::{NotificationClient, ObserverManager}; use risingwave_hummock_sdk::key::{ is_empty_key_range, vnode, vnode_range, TableKey, TableKeyRange, }; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_watermark::TableWatermarksIndex; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::HummockReadEpoch; -use risingwave_pb::hummock::SstableInfo; use risingwave_rpc_client::HummockMetaClient; use thiserror_ext::AsReport; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index f5b27d293bb3..ae0d775219c5 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -24,8 +24,8 @@ use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::MAX_SPILL_TIMES; use risingwave_hummock_sdk::key::{is_empty_key_range, vnode_range, TableKey, TableKeyRange}; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::{EpochWithGap, HummockEpoch}; -use risingwave_pb::hummock::SstableInfo; use tracing::{warn, Instrument}; use super::version::{StagingData, VersionUpdate}; diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 87114f83c7d0..4733b961a37d 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -27,15 +27,17 @@ use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::MAX_SPILL_TIMES; +use risingwave_hummock_sdk::change_log::EpochNewChangeLog; use risingwave_hummock_sdk::key::{ bound_table_key_range, FullKey, TableKey, TableKeyRange, UserKey, }; use risingwave_hummock_sdk::key_range::KeyRangeCommon; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_watermark::{ TableWatermarksIndex, VnodeWatermark, WatermarkDirection, }; use risingwave_hummock_sdk::{EpochWithGap, HummockEpoch, LocalSstableInfo}; -use risingwave_pb::hummock::{EpochNewChangeLog, LevelType, SstableInfo}; +use risingwave_pb::hummock::LevelType; use sync_point::sync_point; use tracing::warn; @@ -612,7 +614,7 @@ impl HummockVersionReader { continue; } - match level.level_type() { + match level.level_type { LevelType::Overlapping | LevelType::Unspecified => { let sstable_infos = prune_overlapping_ssts( &level.table_infos, @@ -648,8 +650,6 @@ impl HummockVersionReader { table_info_idx = table_info_idx.saturating_sub(1); let ord = level.table_infos[table_info_idx] .key_range - .as_ref() - .unwrap() .compare_right_with_user_key(full_key.user_key.as_ref()); // the case that the key falls into the gap between two ssts if ord == Ordering::Less { @@ -862,7 +862,7 @@ impl HummockVersionReader { continue; } - if level.level_type == LevelType::Nonoverlapping as i32 { + if level.level_type == LevelType::Nonoverlapping { let table_infos = prune_nonoverlapping_ssts(&level.table_infos, user_key_range_ref); let sstables = table_infos .filter(|sstable_info| { @@ -927,7 +927,7 @@ impl HummockVersionReader { .sstable_store .sstable(sstable_info, local_stats) .await?; - assert_eq!(sstable_info.get_object_id(), sstable.id); + assert_eq!(sstable_info.object_id, sstable.id); if let Some(dist_hash) = bloom_filter_prefix_hash.as_ref() { if !hit_sstable_bloom_filter( &sstable, diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index cfed5f0eb782..53b58d895c80 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -27,8 +27,9 @@ use risingwave_common::config::EvictionConfig; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::key::{FullKey, PointRange, TableKey, UserKey}; +use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::{EpochWithGap, HummockEpoch, HummockSstableObjectId}; -use risingwave_pb::hummock::{KeyRange, SstableInfo}; use super::iterator::test_utils::iterator_test_table_key_of; use super::{ @@ -112,11 +113,11 @@ pub fn gen_dummy_sst_info( SstableInfo { object_id: id, sst_id: id, - key_range: Some(KeyRange { - left: FullKey::for_test(table_id, min_table_key, epoch).encode(), - right: FullKey::for_test(table_id, max_table_key, epoch).encode(), + key_range: KeyRange { + left: Bytes::from(FullKey::for_test(table_id, min_table_key, epoch).encode()), + right: Bytes::from(FullKey::for_test(table_id, max_table_key, epoch).encode()), right_exclusive: false, - }), + }, file_size, table_ids: vec![table_id.table_id], uncompressed_file_size: file_size, @@ -189,11 +190,11 @@ pub async fn put_sst( let sst = SstableInfo { object_id: sst_object_id, sst_id: sst_object_id, - key_range: Some(KeyRange { - left: meta.smallest_key.clone(), - right: meta.largest_key.clone(), + key_range: KeyRange { + left: Bytes::from(meta.smallest_key.clone()), + right: Bytes::from(meta.largest_key.clone()), right_exclusive: false, - }), + }, file_size: meta.estimated_size as u64, meta_offset: meta.meta_offset, uncompressed_file_size: meta.estimated_size as u64, diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 4c270ee736b9..1e1b23096402 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -28,9 +28,9 @@ use risingwave_common::catalog::{TableId, TableOption}; use risingwave_hummock_sdk::key::{ bound_table_key_range, EmptySliceRef, FullKey, TableKey, UserKey, }; +use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{can_concat, HummockEpoch}; -use risingwave_pb::hummock::SstableInfo; use tokio::sync::oneshot::{channel, Receiver, Sender}; use super::{HummockError, HummockResult}; @@ -88,34 +88,14 @@ pub fn validate_safe_epoch( Ok(()) } -pub fn validate_table_key_range(version: &HummockVersion) { - for l in version.levels.values().flat_map(|levels| { - levels - .l0 - .as_ref() - .unwrap() - .sub_levels - .iter() - .chain(levels.levels.iter()) - }) { - for t in &l.table_infos { - assert!( - t.key_range.is_some(), - "key_range in table [{}] is none", - t.get_object_id() - ); - } - } -} - pub fn filter_single_sst(info: &SstableInfo, table_id: TableId, table_key_range: &R) -> bool where R: RangeBounds>, B: AsRef<[u8]> + EmptySliceRef, { - let table_range = info.key_range.as_ref().unwrap(); - let table_start = FullKey::decode(table_range.left.as_slice()).user_key; - let table_end = FullKey::decode(table_range.right.as_slice()).user_key; + let table_range = &info.key_range; + let table_start = FullKey::decode(table_range.left.as_ref()).user_key; + let table_end = FullKey::decode(table_range.right.as_ref()).user_key; let (left, right) = bound_table_key_range(table_id, table_key_range); let left: Bound> = left.as_ref().map(|key| key.as_ref()); let right: Bound> = right.as_ref().map(|key| key.as_ref()); @@ -127,18 +107,13 @@ where } else { Bound::Included(&table_end) }, - ) && info - .get_table_ids() - .binary_search(&table_id.table_id()) - .is_ok() + ) && info.table_ids.binary_search(&table_id.table_id()).is_ok() } /// Search the SST containing the specified key within a level, using binary search. pub(crate) fn search_sst_idx(ssts: &[SstableInfo], key: UserKey<&[u8]>) -> usize { ssts.partition_point(|table| { - let ord = FullKey::decode(&table.key_range.as_ref().unwrap().left) - .user_key - .cmp(&key); + let ord = FullKey::decode(&table.key_range.left).user_key.cmp(&key); ord == Ordering::Less || ord == Ordering::Equal }) } diff --git a/src/storage/src/hummock/validator.rs b/src/storage/src/hummock/validator.rs index 8c38a99f4324..cc95b7089b66 100644 --- a/src/storage/src/hummock/validator.rs +++ b/src/storage/src/hummock/validator.rs @@ -17,8 +17,8 @@ use std::cmp; use std::collections::HashMap; use std::sync::Arc; +use risingwave_hummock_sdk::compact_task::ValidationTask; use risingwave_hummock_sdk::key::FullKey; -use risingwave_pb::hummock::ValidationTask; use crate::hummock::iterator::HummockIterator; use crate::hummock::sstable::SstableIteratorReadOptions; @@ -40,7 +40,7 @@ pub async fn validate_ssts(task: ValidationTask, sstable_store: SstableStoreRef) .expect("valid worker_id"); tracing::debug!( "Validating SST {} from worker {}, epoch {}", - sst.get_object_id(), + sst.object_id, worker_id, task.epoch ); @@ -48,7 +48,7 @@ pub async fn validate_ssts(task: ValidationTask, sstable_store: SstableStoreRef) Ok(holder) => holder, Err(_err) => { // One reasonable cause is the SST has been vacuumed. - tracing::info!("Skip sanity check for SST {}.", sst.get_object_id()); + tracing::info!("Skip sanity check for SST {}.", sst.object_id); continue; } }; @@ -67,7 +67,7 @@ pub async fn validate_ssts(task: ValidationTask, sstable_store: SstableStoreRef) ); let mut previous_key: Option>> = None; if let Err(_err) = iter.rewind().await { - tracing::info!("Skip sanity check for SST {}.", sst.get_object_id()); + tracing::info!("Skip sanity check for SST {}.", sst.object_id); } while iter.is_valid() { key_counts += 1; @@ -78,38 +78,32 @@ pub async fn validate_ssts(task: ValidationTask, sstable_store: SstableStoreRef) { panic!("SST sanity check failed: Duplicate key {:x?} in SST object {} from worker {} and SST object {} from worker {}", current_key, - sst.get_object_id(), + sst.object_id, worker_id, duplicate_sst_object_id, duplicate_worker_id) } - visited_keys.insert(current_key.to_owned(), (sst.get_object_id(), worker_id)); + visited_keys.insert(current_key.to_owned(), (sst.object_id, worker_id)); // Ordered and Locally unique if let Some(previous_key) = previous_key.take() { let cmp = previous_key.cmp(¤t_key); if cmp != cmp::Ordering::Less { panic!( "SST sanity check failed: For SST {}, expect {:x?} < {:x?}, got {:#?}", - sst.get_object_id(), - previous_key, - current_key, - cmp + sst.object_id, previous_key, current_key, cmp ) } } previous_key = Some(current_key); if let Err(_err) = iter.next().await { - tracing::info!( - "Skip remaining sanity check for SST {}", - sst.get_object_id(), - ); + tracing::info!("Skip remaining sanity check for SST {}", sst.object_id,); break; } } tracing::debug!( "Validated {} keys for SST {}, epoch {}", key_counts, - sst.get_object_id(), + sst.object_id, task.epoch ); iter.collect_local_statistic(&mut unused); diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index b48b793e828a..fbff01658944 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -37,6 +37,7 @@ use risingwave_hummock_trace::{ TracedInitOptions, TracedNewLocalOptions, TracedOpConsistencyLevel, TracedPrefetchOptions, TracedReadOptions, TracedSealCurrentEpochOptions, TracedWriteOptions, }; +use risingwave_pb::hummock::PbVnodeWatermark; use crate::error::{StorageError, StorageResult}; use crate::hummock::CachePolicy; @@ -777,8 +778,11 @@ impl From for TracedSealCurrentEpochOptions { ( direction == WatermarkDirection::Ascending, watermarks - .iter() - .map(|watermark| Message::encode_to_vec(&watermark.to_protobuf())) + .into_iter() + .map(|watermark| { + let pb_watermark = PbVnodeWatermark::from(watermark); + Message::encode_to_vec(&pb_watermark) + }) .collect(), ) }), @@ -800,10 +804,10 @@ impl From for SealCurrentEpochOptions { WatermarkDirection::Descending }, watermarks - .iter() + .into_iter() .map(|serialized_watermark| { Message::decode(serialized_watermark.as_slice()) - .map(|pb| VnodeWatermark::from_protobuf(&pb)) + .map(|pb: PbVnodeWatermark| VnodeWatermark::from(pb)) .expect("should not failed") }) .collect(), diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 167997fae340..add11b2385e2 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -625,7 +625,7 @@ impl LocalBarrierWorker { sst_info, table_stats, }| GroupedSstableInfo { - sst: Some(sst_info), + sst: Some(sst_info.into()), table_stats_map: to_prost_table_stats_map(table_stats), }, ) @@ -633,11 +633,11 @@ impl LocalBarrierWorker { worker_id: self.actor_manager.env.worker_id(), table_watermarks: table_watermarks .into_iter() - .map(|(key, value)| (key.table_id, value.to_protobuf())) + .map(|(key, value)| (key.table_id, value.into())) .collect(), old_value_sstables: old_value_ssts .into_iter() - .map(|sst| sst.sst_info) + .map(|sst| sst.sst_info.into()) .collect(), }, ),