Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

refactor(storage): trigger uploader sync on SyncEpoch event #17113

Merged
merged 16 commits into from
Jun 6, 2024
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion docker/dashboards/risingwave-dev-dashboard.json

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion docker/dashboards/risingwave-user-dashboard.json

Large diffs are not rendered by default.

17 changes: 17 additions & 0 deletions grafana/risingwave-dev-dashboard.dashboard.py
Original file line number Diff line number Diff line change
Expand Up @@ -2270,6 +2270,23 @@ def section_hummock_write(outer_panels):
"uploading task size - {{%s}} @ {{%s}}"
% (COMPONENT_LABEL, NODE_LABEL),
),
panels.target(
f"sum({metric('state_store_uploader_imm_size')}) by ({COMPONENT_LABEL}, {NODE_LABEL})",
"uploader imm size - {{%s}} @ {{%s}}"
% (COMPONENT_LABEL, NODE_LABEL),
),
panels.target(
f"sum({metric('state_store_uploader_imm_size')}) by ({COMPONENT_LABEL}, {NODE_LABEL}) - "
f"sum({metric('state_store_uploader_uploading_task_size')}) by ({COMPONENT_LABEL}, {NODE_LABEL})",
"unflushed imm size - {{%s}} @ {{%s}}"
% (COMPONENT_LABEL, NODE_LABEL),
),
panels.target(
f"sum({metric('uploading_memory_size')}) by ({COMPONENT_LABEL}, {NODE_LABEL}) - "
f"sum({metric('state_store_uploader_imm_size')}) by ({COMPONENT_LABEL}, {NODE_LABEL})",
"orphan imm size - {{%s}} @ {{%s}}"
% (COMPONENT_LABEL, NODE_LABEL),
),
panels.target(
f"sum({metric('state_store_old_value_size')}) by ({COMPONENT_LABEL}, {NODE_LABEL})",
"old value size - {{%s}} @ {{%s}}"
Expand Down
2 changes: 1 addition & 1 deletion grafana/risingwave-dev-dashboard.json

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion grafana/risingwave-user-dashboard.json

Large diffs are not rendered by default.

4 changes: 2 additions & 2 deletions src/storage/benches/bench_imm_compact.rs
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ fn criterion_benchmark(c: &mut Criterion) {
&batches,
|b, batches| {
b.to_async(FuturesExecutor).iter(|| async {
let imm = merge_imms_in_memory(TableId::default(), 0, batches.clone(), None).await;
let imm = merge_imms_in_memory(TableId::default(), batches.clone(), None).await;
assert_eq!(imm.key_count(), 10000 * 100);
assert_eq!(imm.value_count(), 10000 * 100);
})
Expand All @@ -72,7 +72,7 @@ fn criterion_benchmark(c: &mut Criterion) {
&later_batches,
|b, batches| {
b.to_async(FuturesExecutor).iter(|| async {
let imm = merge_imms_in_memory(TableId::default(), 0, batches.clone(), None).await;
let imm = merge_imms_in_memory(TableId::default(), batches.clone(), None).await;
assert_eq!(imm.key_count(), 2000 * 100);
assert_eq!(imm.value_count(), 2000 * 100 * 5);
})
Expand Down
13 changes: 5 additions & 8 deletions src/storage/src/hummock/compactor/shared_buffer_compact.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,7 @@ use crate::filter_key_extractor::{FilterKeyExtractorImpl, FilterKeyExtractorMana
use crate::hummock::compactor::compaction_filter::DummyCompactionFilter;
use crate::hummock::compactor::context::{await_tree_key, CompactorContext};
use crate::hummock::compactor::{check_flush_result, CompactOutput, Compactor};
use crate::hummock::event_handler::uploader::{UploadTaskOutput, UploadTaskPayload};
use crate::hummock::event_handler::LocalInstanceId;
use crate::hummock::event_handler::uploader::UploadTaskOutput;
use crate::hummock::iterator::{Forward, HummockIterator, MergeIterator, UserIterator};
use crate::hummock::shared_buffer::shared_buffer_batch::{
SharedBufferBatch, SharedBufferBatchInner, SharedBufferBatchOldValues, SharedBufferKeyEntry,
Expand All @@ -59,11 +58,11 @@ const GC_WATERMARK_FOR_FLUSH: u64 = 0;
pub async fn compact(
context: CompactorContext,
sstable_object_id_manager: SstableObjectIdManagerRef,
payload: UploadTaskPayload,
payload: Vec<ImmutableMemtable>,
compaction_group_index: Arc<HashMap<TableId, CompactionGroupId>>,
filter_key_extractor_manager: FilterKeyExtractorManager,
) -> HummockResult<UploadTaskOutput> {
let mut grouped_payload: HashMap<CompactionGroupId, UploadTaskPayload> = HashMap::new();
let mut grouped_payload: HashMap<CompactionGroupId, Vec<ImmutableMemtable>> = HashMap::new();
for imm in &payload {
let compaction_group_id = match compaction_group_index.get(&imm.table_id) {
// compaction group id is used only as a hint for grouping different data.
Expand Down Expand Up @@ -144,7 +143,7 @@ async fn compact_shared_buffer<const IS_NEW_VALUE: bool>(
context: CompactorContext,
sstable_object_id_manager: SstableObjectIdManagerRef,
filter_key_extractor_manager: FilterKeyExtractorManager,
mut payload: UploadTaskPayload,
mut payload: Vec<ImmutableMemtable>,
) -> HummockResult<Vec<LocalSstableInfo>> {
if !IS_NEW_VALUE {
assert!(payload.iter().all(|imm| imm.has_old_value()));
Expand Down Expand Up @@ -321,7 +320,6 @@ async fn compact_shared_buffer<const IS_NEW_VALUE: bool>(
/// Merge multiple batches into a larger one
pub async fn merge_imms_in_memory(
table_id: TableId,
instance_id: LocalInstanceId,
imms: Vec<ImmutableMemtable>,
memory_tracker: Option<MemoryTracker>,
) -> ImmutableMemtable {
Expand Down Expand Up @@ -456,13 +454,12 @@ pub async fn merge_imms_in_memory(
memory_tracker,
)),
table_id,
instance_id,
}
}

/// Based on the incoming payload and opts, calculate the sharding method and sstable size of shared buffer compaction.
fn generate_splits(
payload: &UploadTaskPayload,
payload: &Vec<ImmutableMemtable>,
existing_table_ids: &HashSet<u32>,
storage_opts: &StorageOpts,
) -> (Vec<KeyRange>, u64, BTreeMap<u32, u32>) {
Expand Down
Loading
Loading