1use std::net::SocketAddr;
16use std::sync::Arc;
17use std::time::Duration;
18
19use risingwave_common::config::{
20 AsyncStackTraceOption, MetricLevel, RwConfig, extract_storage_memory_config, load_config,
21};
22use risingwave_common::monitor::{RouterExt, TcpConfig};
23use risingwave_common::system_param::local_manager::LocalSystemParamsManager;
24use risingwave_common::system_param::reader::{SystemParamsRead, SystemParamsReader};
25use risingwave_common::telemetry::manager::TelemetryManager;
26use risingwave_common::telemetry::telemetry_env_enabled;
27use risingwave_common::util::addr::HostAddr;
28use risingwave_common::util::resource_util::memory::system_memory_available_bytes;
29use risingwave_common::util::tokio_util::sync::CancellationToken;
30use risingwave_common::{GIT_SHA, RW_VERSION};
31use risingwave_common_heap_profiling::HeapProfiler;
32use risingwave_common_service::{MetricsManager, ObserverManager};
33use risingwave_object_store::object::build_remote_object_store;
34use risingwave_object_store::object::object_metrics::GLOBAL_OBJECT_STORE_METRICS;
35use risingwave_pb::common::WorkerType;
36use risingwave_pb::compactor::compactor_service_server::CompactorServiceServer;
37use risingwave_pb::monitor_service::monitor_service_server::MonitorServiceServer;
38use risingwave_rpc_client::{GrpcCompactorProxyClient, MetaClient};
39use risingwave_storage::compaction_catalog_manager::{
40 CompactionCatalogManager, RemoteTableAccessor,
41};
42use risingwave_storage::hummock::compactor::{
43 CompactionAwaitTreeRegRef, CompactionExecutor, CompactorContext,
44 new_compaction_await_tree_reg_ref,
45};
46use risingwave_storage::hummock::hummock_meta_client::MonitoredHummockMetaClient;
47use risingwave_storage::hummock::utils::HummockMemoryCollector;
48use risingwave_storage::hummock::{MemoryLimiter, SstableObjectIdManager, SstableStore};
49use risingwave_storage::monitor::{
50 CompactorMetrics, GLOBAL_COMPACTOR_METRICS, GLOBAL_HUMMOCK_METRICS, monitor_cache,
51};
52use risingwave_storage::opts::StorageOpts;
53use tokio::sync::mpsc;
54use tracing::info;
55
56use super::compactor_observer::observer_manager::CompactorObserverNode;
57use crate::CompactorOpts;
58use crate::rpc::{CompactorServiceImpl, MonitorServiceImpl};
59use crate::telemetry::CompactorTelemetryCreator;
60
61pub async fn prepare_start_parameters(
62 compactor_opts: &CompactorOpts,
63 config: RwConfig,
64 system_params_reader: SystemParamsReader,
65) -> (
66 Arc<SstableStore>,
67 Arc<MemoryLimiter>,
68 HeapProfiler,
69 Option<CompactionAwaitTreeRegRef>,
70 Arc<StorageOpts>,
71 Arc<CompactorMetrics>,
72) {
73 let object_metrics = Arc::new(GLOBAL_OBJECT_STORE_METRICS.clone());
75 let compactor_metrics = Arc::new(GLOBAL_COMPACTOR_METRICS.clone());
76
77 let state_store_url = system_params_reader.state_store();
78
79 let storage_memory_config = extract_storage_memory_config(&config);
80 let storage_opts: Arc<StorageOpts> = Arc::new(StorageOpts::from((
81 &config,
82 &system_params_reader,
83 &storage_memory_config,
84 )));
85 let non_reserved_memory_bytes = (compactor_opts.compactor_total_memory_bytes as f64
86 * config.storage.compactor_memory_available_proportion)
87 as usize;
88 let meta_cache_capacity_bytes = compactor_opts.compactor_meta_cache_memory_bytes;
89 let compactor_memory_limit_bytes = match config.storage.compactor_memory_limit_mb {
90 Some(compactor_memory_limit_mb) => compactor_memory_limit_mb * (1 << 20),
91 None => {
92 non_reserved_memory_bytes
93 }
94 }
95 .checked_sub(compactor_opts.compactor_meta_cache_memory_bytes).unwrap_or_else(|| {
96 panic!(
97 "compactor_total_memory_bytes {} is too small to hold compactor_meta_cache_memory_bytes {}",
98 meta_cache_capacity_bytes,
99 meta_cache_capacity_bytes
100 );
101 });
102
103 tracing::info!(
104 "Compactor non_reserved_memory_bytes {} meta_cache_capacity_bytes {} compactor_memory_limit_bytes {} sstable_size_bytes {} block_size_bytes {}",
105 non_reserved_memory_bytes,
106 meta_cache_capacity_bytes,
107 compactor_memory_limit_bytes,
108 storage_opts.sstable_size_mb * (1 << 20),
109 storage_opts.block_size_kb * (1 << 10),
110 );
111
112 {
114 let min_compactor_memory_limit_bytes = (storage_opts.sstable_size_mb * (1 << 20)
117 + storage_opts.block_size_kb * (1 << 10))
118 as u64;
119
120 assert!(compactor_memory_limit_bytes > min_compactor_memory_limit_bytes as usize * 2);
121 }
122
123 let object_store = build_remote_object_store(
124 state_store_url
125 .strip_prefix("hummock+")
126 .expect("object store must be hummock for compactor server"),
127 object_metrics,
128 "Hummock",
129 Arc::new(config.storage.object_store.clone()),
130 )
131 .await;
132
133 let object_store = Arc::new(object_store);
134 let sstable_store = Arc::new(
135 SstableStore::for_compactor(
136 object_store,
137 storage_opts.data_directory.clone(),
138 0,
139 meta_cache_capacity_bytes,
140 system_params_reader.use_new_object_prefix_strategy(),
141 )
142 .await
143 .unwrap(),
145 );
146
147 let memory_limiter = Arc::new(MemoryLimiter::new(compactor_memory_limit_bytes as u64));
148 let storage_memory_config = extract_storage_memory_config(&config);
149 let memory_collector = Arc::new(HummockMemoryCollector::new(
150 sstable_store.clone(),
151 memory_limiter.clone(),
152 storage_memory_config,
153 ));
154
155 let heap_profiler = HeapProfiler::new(
156 system_memory_available_bytes(),
157 config.server.heap_profiling.clone(),
158 );
159
160 monitor_cache(memory_collector);
161
162 let await_tree_config = match &config.streaming.async_stack_trace {
163 AsyncStackTraceOption::Off => None,
164 c => await_tree::ConfigBuilder::default()
165 .verbose(c.is_verbose().unwrap())
166 .build()
167 .ok(),
168 };
169 let await_tree_reg = await_tree_config.map(new_compaction_await_tree_reg_ref);
170
171 (
172 sstable_store,
173 memory_limiter,
174 heap_profiler,
175 await_tree_reg,
176 storage_opts,
177 compactor_metrics,
178 )
179}
180
181pub async fn compactor_serve(
185 listen_addr: SocketAddr,
186 advertise_addr: HostAddr,
187 opts: CompactorOpts,
188 shutdown: CancellationToken,
189) {
190 let config = load_config(&opts.config_path, &opts);
191 info!("Starting compactor node",);
192 info!("> config: {:?}", config);
193 info!(
194 "> debug assertions: {}",
195 if cfg!(debug_assertions) { "on" } else { "off" }
196 );
197 info!("> version: {} ({})", RW_VERSION, GIT_SHA);
198
199 let (meta_client, system_params_reader) = MetaClient::register_new(
201 opts.meta_address.clone(),
202 WorkerType::Compactor,
203 &advertise_addr,
204 Default::default(),
205 &config.meta,
206 )
207 .await;
208
209 info!("Assigned compactor id {}", meta_client.worker_id());
210
211 let hummock_metrics = Arc::new(GLOBAL_HUMMOCK_METRICS.clone());
212
213 let hummock_meta_client = Arc::new(MonitoredHummockMetaClient::new(
214 meta_client.clone(),
215 hummock_metrics.clone(),
216 ));
217
218 let (
219 sstable_store,
220 memory_limiter,
221 heap_profiler,
222 await_tree_reg,
223 storage_opts,
224 compactor_metrics,
225 ) = prepare_start_parameters(&opts, config.clone(), system_params_reader.clone()).await;
226
227 let compaction_catalog_manager_ref = Arc::new(CompactionCatalogManager::new(Box::new(
228 RemoteTableAccessor::new(meta_client.clone()),
229 )));
230
231 let system_params_manager = Arc::new(LocalSystemParamsManager::new(system_params_reader));
232 let compactor_observer_node = CompactorObserverNode::new(
233 compaction_catalog_manager_ref.clone(),
234 system_params_manager.clone(),
235 );
236 let observer_manager =
237 ObserverManager::new_with_meta_client(meta_client.clone(), compactor_observer_node).await;
238
239 heap_profiler.start();
241
242 let _observer_join_handle = observer_manager.start().await;
245
246 let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new(
247 hummock_meta_client.clone(),
248 storage_opts.sstable_id_remote_fetch_number,
249 ));
250
251 let compaction_executor = Arc::new(CompactionExecutor::new(
252 opts.compaction_worker_threads_number,
253 ));
254
255 let compactor_context = CompactorContext {
256 storage_opts,
257 sstable_store: sstable_store.clone(),
258 compactor_metrics,
259 is_share_buffer_compact: false,
260 compaction_executor,
261 memory_limiter,
262 task_progress_manager: Default::default(),
263 await_tree_reg: await_tree_reg.clone(),
264 };
265
266 let mut sub_tasks = vec![
268 MetaClient::start_heartbeat_loop(
269 meta_client.clone(),
270 Duration::from_millis(config.server.heartbeat_interval_ms as u64),
271 ),
272 risingwave_storage::hummock::compactor::start_compactor(
273 compactor_context.clone(),
274 hummock_meta_client.clone(),
275 sstable_object_id_manager.clone(),
276 compaction_catalog_manager_ref,
277 ),
278 ];
279
280 let telemetry_manager = TelemetryManager::new(
281 Arc::new(meta_client.clone()),
282 Arc::new(CompactorTelemetryCreator::new()),
283 );
284 if config.server.telemetry_enabled && telemetry_env_enabled() {
287 sub_tasks.push(telemetry_manager.start().await);
288 } else {
289 tracing::info!("Telemetry didn't start due to config");
290 }
291
292 let compactor_srv = CompactorServiceImpl::default();
293 let monitor_srv = MonitorServiceImpl::new(await_tree_reg);
294 let server = tonic::transport::Server::builder()
295 .add_service(CompactorServiceServer::new(compactor_srv))
296 .add_service(MonitorServiceServer::new(monitor_srv))
297 .monitored_serve_with_shutdown(
298 listen_addr,
299 "grpc-compactor-node-service",
300 TcpConfig {
301 tcp_nodelay: true,
302 keepalive_duration: None,
303 },
304 shutdown.clone().cancelled_owned(),
305 );
306 let _server_handle = tokio::spawn(server);
307
308 if config.server.metrics_level > MetricLevel::Disabled {
310 MetricsManager::boot_metrics_service(opts.prometheus_listener_addr.clone());
311 }
312
313 meta_client.activate(&advertise_addr).await.unwrap();
315
316 shutdown.cancelled().await;
318 meta_client.try_unregister().await;
320}
321
322pub async fn shared_compactor_serve(
326 listen_addr: SocketAddr,
327 opts: CompactorOpts,
328 shutdown: CancellationToken,
329) {
330 let config = load_config(&opts.config_path, &opts);
331 info!("Starting shared compactor node",);
332 info!("> config: {:?}", config);
333 info!(
334 "> debug assertions: {}",
335 if cfg!(debug_assertions) { "on" } else { "off" }
336 );
337 info!("> version: {} ({})", RW_VERSION, GIT_SHA);
338
339 let grpc_proxy_client = GrpcCompactorProxyClient::new(opts.proxy_rpc_endpoint.clone()).await;
340 let system_params_response = grpc_proxy_client
341 .get_system_params()
342 .await
343 .expect("Fail to get system params, the compactor pod cannot be started.");
344 let system_params = system_params_response.into_inner().params.unwrap();
345
346 let (
347 sstable_store,
348 memory_limiter,
349 heap_profiler,
350 await_tree_reg,
351 storage_opts,
352 compactor_metrics,
353 ) = prepare_start_parameters(&opts, config.clone(), system_params.into()).await;
354 let (sender, receiver) = mpsc::unbounded_channel();
355 let compactor_srv: CompactorServiceImpl = CompactorServiceImpl::new(sender);
356
357 let monitor_srv = MonitorServiceImpl::new(await_tree_reg.clone());
358
359 heap_profiler.start();
361
362 let compaction_executor = Arc::new(CompactionExecutor::new(
363 opts.compaction_worker_threads_number,
364 ));
365 let compactor_context = CompactorContext {
366 storage_opts,
367 sstable_store,
368 compactor_metrics,
369 is_share_buffer_compact: false,
370 compaction_executor,
371 memory_limiter,
372 task_progress_manager: Default::default(),
373 await_tree_reg,
374 };
375
376 let _compactor_handle = risingwave_storage::hummock::compactor::start_shared_compactor(
379 grpc_proxy_client,
380 receiver,
381 compactor_context,
382 );
383
384 let server = tonic::transport::Server::builder()
385 .add_service(CompactorServiceServer::new(compactor_srv))
386 .add_service(MonitorServiceServer::new(monitor_srv))
387 .monitored_serve_with_shutdown(
388 listen_addr,
389 "grpc-compactor-node-service",
390 TcpConfig {
391 tcp_nodelay: true,
392 keepalive_duration: None,
393 },
394 shutdown.clone().cancelled_owned(),
395 );
396
397 let _server_handle = tokio::spawn(server);
398
399 if config.server.metrics_level > MetricLevel::Disabled {
401 MetricsManager::boot_metrics_service(opts.prometheus_listener_addr.clone());
402 }
403
404 shutdown.cancelled().await;
406
407 }