Skip to content

Commit

Permalink
feat(ctl): support pause/resume hummock version checkpoint (#10069)
Browse files Browse the repository at this point in the history
  • Loading branch information
zwang28 authored May 30, 2023
1 parent f15c315 commit 5013b5e
Show file tree
Hide file tree
Showing 10 changed files with 207 additions and 36 deletions.
17 changes: 17 additions & 0 deletions proto/hummock.proto
Original file line number Diff line number Diff line change
Expand Up @@ -579,6 +579,20 @@ message SplitCompactionGroupResponse {
uint64 new_group_id = 1;
}

message RiseCtlPauseVersionCheckpointRequest {}

message RiseCtlPauseVersionCheckpointResponse {}

message RiseCtlResumeVersionCheckpointRequest {}

message RiseCtlResumeVersionCheckpointResponse {}

message RiseCtlGetCheckpointVersionRequest {}

message RiseCtlGetCheckpointVersionResponse {
HummockVersion checkpoint_version = 1;
}

service HummockManagerService {
rpc UnpinVersionBefore(UnpinVersionBeforeRequest) returns (UnpinVersionBeforeResponse);
rpc GetCurrentVersion(GetCurrentVersionRequest) returns (GetCurrentVersionResponse);
Expand All @@ -604,6 +618,9 @@ service HummockManagerService {
rpc RiseCtlGetPinnedSnapshotsSummary(RiseCtlGetPinnedSnapshotsSummaryRequest) returns (RiseCtlGetPinnedSnapshotsSummaryResponse);
rpc RiseCtlListCompactionGroup(RiseCtlListCompactionGroupRequest) returns (RiseCtlListCompactionGroupResponse);
rpc RiseCtlUpdateCompactionConfig(RiseCtlUpdateCompactionConfigRequest) returns (RiseCtlUpdateCompactionConfigResponse);
rpc RiseCtlPauseVersionCheckpoint(RiseCtlPauseVersionCheckpointRequest) returns (RiseCtlPauseVersionCheckpointResponse);
rpc RiseCtlResumeVersionCheckpoint(RiseCtlResumeVersionCheckpointRequest) returns (RiseCtlResumeVersionCheckpointResponse);
rpc RiseCtlGetCheckpointVersion(RiseCtlGetCheckpointVersionRequest) returns (RiseCtlGetCheckpointVersionResponse);
rpc InitMetadataForReplay(InitMetadataForReplayRequest) returns (InitMetadataForReplayResponse);
rpc GetScaleCompactor(GetScaleCompactorRequest) returns (GetScaleCompactorResponse);
rpc PinVersion(PinVersionRequest) returns (PinVersionResponse);
Expand Down
4 changes: 2 additions & 2 deletions src/ctl/src/cmd_impl/hummock.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,13 @@ pub use list_kv::*;
mod sst_dump;
pub use sst_dump::*;
mod compaction_group;
mod disable_commit_epoch;
mod list_version_deltas;
mod pause_resume;
mod trigger_full_gc;
mod trigger_manual_compaction;

pub use compaction_group::*;
pub use disable_commit_epoch::*;
pub use list_version_deltas::*;
pub use pause_resume::*;
pub use trigger_full_gc::*;
pub use trigger_manual_compaction::*;
26 changes: 0 additions & 26 deletions src/ctl/src/cmd_impl/hummock/disable_commit_epoch.rs

This file was deleted.

83 changes: 83 additions & 0 deletions src/ctl/src/cmd_impl/hummock/pause_resume.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
// Copyright 2023 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 risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionUpdateExt;
use risingwave_hummock_sdk::HummockEpoch;

use crate::CtlContext;

pub async fn disable_commit_epoch(context: &CtlContext) -> anyhow::Result<()> {
let meta_client = context.meta_client().await?;
let version = meta_client.disable_commit_epoch().await?;
println!(
"Disabled.\
Current version: id {}, max_committed_epoch {}",
version.id, version.max_committed_epoch
);
Ok(())
}

pub async fn pause_version_checkpoint(context: &CtlContext) -> anyhow::Result<()> {
let meta_client = context.meta_client().await?;
meta_client
.risectl_pause_hummock_version_checkpoint()
.await?;
println!("Hummock version checkpoint is paused");
Ok(())
}

pub async fn resume_version_checkpoint(context: &CtlContext) -> anyhow::Result<()> {
let meta_client = context.meta_client().await?;
meta_client
.risectl_resume_hummock_version_checkpoint()
.await?;
println!("Hummock version checkpoint is resumed");
Ok(())
}

/// For now this function itself doesn't provide useful info.
/// We can extend it to reveal interested info, e.g. at which hummock version is a user key
/// added/removed for what reason (row deletion/compaction/etc.).
pub async fn replay_version(context: &CtlContext) -> anyhow::Result<()> {
let meta_client = context.meta_client().await?;
let mut base_version = meta_client
.risectl_get_checkpoint_hummock_version()
.await?
.checkpoint_version
.unwrap();
println!("replay starts");
println!("base version {}", base_version.id);
let delta_fetch_size = 100;
let mut current_delta_id = base_version.id + 1;
loop {
let deltas = meta_client
.list_version_deltas(current_delta_id, delta_fetch_size, HummockEpoch::MAX)
.await
.unwrap();
if deltas.version_deltas.is_empty() {
break;
}
for delta in deltas.version_deltas {
if delta.prev_id != base_version.id {
eprintln!("missing delta log for version {}", base_version.id);
break;
}
base_version.apply_version_delta(&delta);
println!("replayed version {}", base_version.id);
}
current_delta_id = base_version.id + 1;
}
println!("replay ends");
Ok(())
}
15 changes: 15 additions & 0 deletions src/ctl/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -158,6 +158,12 @@ enum HummockCommands {
#[clap(long)]
table_ids: Vec<u32>,
},
/// Pause version checkpoint, which subsequently pauses GC of delta log and SST object.
PauseVersionCheckpoint,
/// Resume version checkpoint, which subsequently resumes GC of delta log and SST object.
ResumeVersionCheckpoint,
/// Replay version from the checkpoint one to the latest one.
ReplayVersion,
}

#[derive(Subcommand)]
Expand Down Expand Up @@ -319,6 +325,15 @@ pub async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> {
cmd_impl::hummock::split_compaction_group(context, compaction_group_id, &table_ids)
.await?;
}
Commands::Hummock(HummockCommands::PauseVersionCheckpoint) => {
cmd_impl::hummock::pause_version_checkpoint(context).await?;
}
Commands::Hummock(HummockCommands::ResumeVersionCheckpoint) => {
cmd_impl::hummock::resume_version_checkpoint(context).await?;
}
Commands::Hummock(HummockCommands::ReplayVersion) => {
cmd_impl::hummock::replay_version(context).await?;
}
Commands::Table(TableCommands::Scan { mv_name, data_dir }) => {
cmd_impl::table::scan(context, mv_name, data_dir).await?
}
Expand Down
29 changes: 28 additions & 1 deletion src/meta/src/hummock/manager/checkpoint.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@

use std::ops::Bound::{Excluded, Included};
use std::ops::{Deref, DerefMut};
use std::sync::atomic::Ordering;

use function_name::named;
use itertools::Itertools;
Expand All @@ -22,7 +23,7 @@ use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{
};
use risingwave_hummock_sdk::version_checkpoint_dir;
use risingwave_pb::hummock::hummock_version_checkpoint::StaleObjects;
use risingwave_pb::hummock::HummockVersionCheckpoint;
use risingwave_pb::hummock::{HummockVersion, HummockVersionCheckpoint};

use crate::hummock::error::Result;
use crate::hummock::manager::{read_lock, write_lock};
Expand Down Expand Up @@ -175,4 +176,30 @@ where
.await
.map_err(Into::into)
}

pub(crate) fn pause_version_checkpoint(&self) {
self.pause_version_checkpoint.store(true, Ordering::Relaxed);
tracing::info!("hummock version checkpoint is paused.");
}

pub(crate) fn resume_version_checkpoint(&self) {
self.pause_version_checkpoint
.store(false, Ordering::Relaxed);
tracing::info!("hummock version checkpoint is resumed.");
}

pub(crate) fn is_version_checkpoint_paused(&self) -> bool {
self.pause_version_checkpoint.load(Ordering::Relaxed)
}

#[named]
pub(crate) async fn get_checkpoint_version(&self) -> HummockVersion {
let versioning_guard = read_lock!(self, versioning).await;
versioning_guard
.checkpoint
.version
.as_ref()
.unwrap()
.clone()
}
}
3 changes: 3 additions & 0 deletions src/meta/src/hummock/manager/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ use core::panic;
use std::borrow::BorrowMut;
use std::collections::{BTreeMap, HashMap, HashSet};
use std::ops::{Deref, DerefMut};
use std::sync::atomic::AtomicBool;
use std::sync::{Arc, LazyLock};
use std::time::{Duration, Instant};

Expand Down Expand Up @@ -119,6 +120,7 @@ pub struct HummockManager<S: MetaStore> {

object_store: ObjectStoreRef,
version_checkpoint_path: String,
pause_version_checkpoint: AtomicBool,
}

pub type HummockManagerRef<S> = Arc<HummockManager<S>>;
Expand Down Expand Up @@ -342,6 +344,7 @@ where
event_sender: tx,
object_store,
version_checkpoint_path: checkpoint_path,
pause_version_checkpoint: AtomicBool::new(false),
};
let instance = Arc::new(instance);
instance.start_worker(rx).await;
Expand Down
16 changes: 9 additions & 7 deletions src/meta/src/hummock/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -64,13 +64,10 @@ where
meta_opts.min_delta_log_num_for_hummock_version_checkpoint,
),
];
// Start vacuum in non-deterministic compaction test
if !meta_opts.compaction_deterministic_test {
workers.push(start_vacuum_scheduler(
vacuum_manager,
Duration::from_secs(meta_opts.vacuum_interval_sec),
));
}
workers.push(start_vacuum_scheduler(
vacuum_manager,
Duration::from_secs(meta_opts.vacuum_interval_sec),
));
workers
}

Expand Down Expand Up @@ -145,6 +142,11 @@ pub fn start_checkpoint_loop<S: MetaStore>(
return;
}
}
if hummock_manager.is_version_checkpoint_paused()
|| hummock_manager.env.opts.compaction_deterministic_test
{
continue;
}
if let Err(err) = hummock_manager
.create_version_checkpoint(min_delta_log_num)
.await
Expand Down
26 changes: 26 additions & 0 deletions src/meta/src/rpc/service/hummock_service.rs
Original file line number Diff line number Diff line change
Expand Up @@ -515,4 +515,30 @@ where
resp.suggest_cores = scale_out_cores;
Ok(Response::new(resp))
}

async fn rise_ctl_pause_version_checkpoint(
&self,
_request: Request<RiseCtlPauseVersionCheckpointRequest>,
) -> Result<Response<RiseCtlPauseVersionCheckpointResponse>, Status> {
self.hummock_manager.pause_version_checkpoint();
Ok(Response::new(RiseCtlPauseVersionCheckpointResponse {}))
}

async fn rise_ctl_resume_version_checkpoint(
&self,
_request: Request<RiseCtlResumeVersionCheckpointRequest>,
) -> Result<Response<RiseCtlResumeVersionCheckpointResponse>, Status> {
self.hummock_manager.resume_version_checkpoint();
Ok(Response::new(RiseCtlResumeVersionCheckpointResponse {}))
}

async fn rise_ctl_get_checkpoint_version(
&self,
_request: Request<RiseCtlGetCheckpointVersionRequest>,
) -> Result<Response<RiseCtlGetCheckpointVersionResponse>, Status> {
let checkpoint_version = self.hummock_manager.get_checkpoint_version().await;
Ok(Response::new(RiseCtlGetCheckpointVersionResponse {
checkpoint_version: Some(checkpoint_version),
}))
}
}
24 changes: 24 additions & 0 deletions src/rpc_client/src/meta_client.rs
Original file line number Diff line number Diff line change
Expand Up @@ -691,6 +691,27 @@ impl MetaClient {
.await
}

pub async fn risectl_get_checkpoint_hummock_version(
&self,
) -> Result<RiseCtlGetCheckpointVersionResponse> {
let request = RiseCtlGetCheckpointVersionRequest {};
self.inner.rise_ctl_get_checkpoint_version(request).await
}

pub async fn risectl_pause_hummock_version_checkpoint(
&self,
) -> Result<RiseCtlPauseVersionCheckpointResponse> {
let request = RiseCtlPauseVersionCheckpointRequest {};
self.inner.rise_ctl_pause_version_checkpoint(request).await
}

pub async fn risectl_resume_hummock_version_checkpoint(
&self,
) -> Result<RiseCtlResumeVersionCheckpointResponse> {
let request = RiseCtlResumeVersionCheckpointRequest {};
self.inner.rise_ctl_resume_version_checkpoint(request).await
}

pub async fn init_metadata_for_replay(
&self,
tables: Vec<PbTable>,
Expand Down Expand Up @@ -1466,6 +1487,9 @@ macro_rules! for_all_meta_rpc {
,{ hummock_client, rise_ctl_get_pinned_snapshots_summary, RiseCtlGetPinnedSnapshotsSummaryRequest, RiseCtlGetPinnedSnapshotsSummaryResponse }
,{ hummock_client, rise_ctl_list_compaction_group, RiseCtlListCompactionGroupRequest, RiseCtlListCompactionGroupResponse }
,{ hummock_client, rise_ctl_update_compaction_config, RiseCtlUpdateCompactionConfigRequest, RiseCtlUpdateCompactionConfigResponse }
,{ hummock_client, rise_ctl_get_checkpoint_version, RiseCtlGetCheckpointVersionRequest, RiseCtlGetCheckpointVersionResponse }
,{ hummock_client, rise_ctl_pause_version_checkpoint, RiseCtlPauseVersionCheckpointRequest, RiseCtlPauseVersionCheckpointResponse }
,{ hummock_client, rise_ctl_resume_version_checkpoint, RiseCtlResumeVersionCheckpointRequest, RiseCtlResumeVersionCheckpointResponse }
,{ hummock_client, init_metadata_for_replay, InitMetadataForReplayRequest, InitMetadataForReplayResponse }
,{ hummock_client, split_compaction_group, SplitCompactionGroupRequest, SplitCompactionGroupResponse }
,{ user_client, create_user, CreateUserRequest, CreateUserResponse }
Expand Down

0 comments on commit 5013b5e

Please sign in to comment.