-
Notifications
You must be signed in to change notification settings - Fork 596
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat(storage): support online cache resize via risectl #19677
Changes from 7 commits
1592342
2f1ae91
b31acd3
c7a8ebc
291220f
fb31a47
40487ed
23d0158
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -13,16 +13,24 @@ | |
// limitations under the License. | ||
use std::sync::Arc; | ||
|
||
use foyer::HybridCache; | ||
use risingwave_batch::task::BatchManager; | ||
use risingwave_common::error::tonic::ToTonicStatus; | ||
use risingwave_hummock_sdk::HummockSstableObjectId; | ||
use risingwave_pb::compute::config_service_server::ConfigService; | ||
use risingwave_pb::compute::{ShowConfigRequest, ShowConfigResponse}; | ||
use risingwave_pb::compute::{ | ||
ResizeCacheRequest, ResizeCacheResponse, ShowConfigRequest, ShowConfigResponse, | ||
}; | ||
use risingwave_storage::hummock::{Block, Sstable, SstableBlockIndex}; | ||
use risingwave_stream::task::LocalStreamManager; | ||
use thiserror_ext::AsReport; | ||
use tonic::{Code, Request, Response, Status}; | ||
|
||
pub struct ConfigServiceImpl { | ||
batch_mgr: Arc<BatchManager>, | ||
stream_mgr: LocalStreamManager, | ||
meta_cache: Option<HybridCache<HummockSstableObjectId, Box<Sstable>>>, | ||
block_cache: Option<HybridCache<SstableBlockIndex, Box<Block>>>, | ||
} | ||
|
||
#[async_trait::async_trait] | ||
|
@@ -42,13 +50,53 @@ impl ConfigService for ConfigServiceImpl { | |
}; | ||
Ok(Response::new(show_config_response)) | ||
} | ||
|
||
async fn resize_cache( | ||
&self, | ||
request: Request<ResizeCacheRequest>, | ||
) -> Result<Response<ResizeCacheResponse>, Status> { | ||
let req = request.into_inner(); | ||
|
||
if let Some(meta_cache) = &self.meta_cache | ||
&& req.meta_cache_capacity > 0 | ||
{ | ||
match meta_cache.memory().resize(req.meta_cache_capacity as _) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. IIUC, the resize operation is not persistent, which means if the CN restarts, the cache capacity will be back to the configured values from the toml config. Will this be a problem? For example, if we have 2 CNs in the cluster and after resize_cache, 1 CN restarts for some reason while the other one doesn't. I think it depends on when we are going to use resize_cache. If it is mainly for testing or perf tunning and after the tuning we will update the configs, this is fine. But if we rely on resize_cache in production, that will not be ideal. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it depends on how we define resize. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In fact, I also think that multi-CN cache config inconsistencies are a concern, and it's not easy to detect. If we allow online resize cache size, then we need to make sure that the operation succeeds on all machines, and is persistent, otherwise this is a risk. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
This feature just aims to resize the in-memory meta/data block cache without downtime. If there are restarts, it is fine to modify the persistent configuration directly. Modifying the per-node configuration can be (and maybe better be) achieved in the cloud control panel. |
||
Ok(_) => tracing::info!( | ||
"resize meta cache capacity to {:?}", | ||
req.meta_cache_capacity | ||
), | ||
Err(e) => return Err(Status::internal(e.to_report_string())), | ||
} | ||
} | ||
|
||
if let Some(block_cache) = &self.block_cache | ||
&& req.data_cache_capacity > 0 | ||
{ | ||
match block_cache.memory().resize(req.data_cache_capacity as _) { | ||
Ok(_) => tracing::info!( | ||
"resize data cache capacity to {:?}", | ||
req.data_cache_capacity | ||
), | ||
Err(e) => return Err(Status::internal(e.to_report_string())), | ||
} | ||
} | ||
|
||
Ok(Response::new(ResizeCacheResponse {})) | ||
} | ||
} | ||
|
||
impl ConfigServiceImpl { | ||
pub fn new(batch_mgr: Arc<BatchManager>, stream_mgr: LocalStreamManager) -> Self { | ||
pub fn new( | ||
batch_mgr: Arc<BatchManager>, | ||
stream_mgr: LocalStreamManager, | ||
meta_cache: Option<HybridCache<HummockSstableObjectId, Box<Sstable>>>, | ||
block_cache: Option<HybridCache<SstableBlockIndex, Box<Block>>>, | ||
) -> Self { | ||
Self { | ||
batch_mgr, | ||
stream_mgr, | ||
meta_cache, | ||
block_cache, | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
// 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::process::exit; | ||
|
||
use futures::future::try_join_all; | ||
use risingwave_pb::compute::ResizeCacheRequest; | ||
use risingwave_pb::meta::GetClusterInfoResponse; | ||
use risingwave_rpc_client::ComputeClient; | ||
use thiserror_ext::AsReport; | ||
|
||
use crate::common::CtlContext; | ||
|
||
macro_rules! fail { | ||
($($arg:tt)*) => {{ | ||
println!($($arg)*); | ||
exit(1); | ||
}}; | ||
} | ||
|
||
pub async fn resize_cache( | ||
context: &CtlContext, | ||
meta_cache_capacity: Option<u64>, | ||
data_cache_capacity: Option<u64>, | ||
) -> anyhow::Result<()> { | ||
let meta_client = context.meta_client().await?; | ||
|
||
let GetClusterInfoResponse { worker_nodes, .. } = match meta_client.get_cluster_info().await { | ||
Ok(resp) => resp, | ||
Err(e) => { | ||
fail!("Failed to get cluster info: {}", e.as_report()); | ||
} | ||
}; | ||
|
||
let futures = worker_nodes.iter().map(|worker| async { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. just a question: I'm considering a partial success. Do we need to provide a retry capability to perform an rpc retry for the wrong worker node, to avoid as much as possible inconsistencies in the configs of multiple CNs? (I believe this is an idempotent operation.) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. IMO, it is okay to make the user responsible to retry it. |
||
let addr = worker.get_host().expect("worker host must be set"); | ||
let client = ComputeClient::new(addr.into()) | ||
.await | ||
.unwrap_or_else(|_| panic!("Cannot open client to compute node {addr:?}")); | ||
client | ||
.resize_cache(ResizeCacheRequest { | ||
meta_cache_capacity: meta_cache_capacity.unwrap_or(0), | ||
data_cache_capacity: data_cache_capacity.unwrap_or(0), | ||
}) | ||
.await | ||
}); | ||
|
||
if let Err(e) = try_join_all(futures).await { | ||
fail!("Failed to resize cache: {}", e.as_report()) | ||
} | ||
|
||
Ok(()) | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
get_meta_cache_memory_usage_ratio and get_block_cache_memory_usage_ratio will be inaccurate after this PR. We should change
HummockMemoryCollector
to usecache.capacity()
instead.