risingwave_compute/
server.rs

1// Copyright 2022 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::net::SocketAddr;
16use std::sync::Arc;
17use std::time::Duration;
18
19use risingwave_batch::monitor::{
20    GLOBAL_BATCH_EXECUTOR_METRICS, GLOBAL_BATCH_MANAGER_METRICS, GLOBAL_BATCH_SPILL_METRICS,
21};
22use risingwave_batch::rpc::service::task_service::BatchServiceImpl;
23use risingwave_batch::spill::spill_op::SpillOp;
24use risingwave_batch::task::{BatchEnvironment, BatchManager};
25use risingwave_common::config::{
26    AsyncStackTraceOption, MAX_CONNECTION_WINDOW_SIZE, MetricLevel, STREAM_WINDOW_SIZE,
27    StorageMemoryConfig, load_config,
28};
29use risingwave_common::license::LicenseManager;
30use risingwave_common::lru::init_global_sequencer_args;
31use risingwave_common::monitor::{RouterExt, TcpConfig};
32use risingwave_common::secret::LocalSecretManager;
33use risingwave_common::system_param::local_manager::LocalSystemParamsManager;
34use risingwave_common::system_param::reader::SystemParamsRead;
35use risingwave_common::telemetry::manager::TelemetryManager;
36use risingwave_common::telemetry::telemetry_env_enabled;
37use risingwave_common::util::addr::HostAddr;
38use risingwave_common::util::pretty_bytes::convert;
39use risingwave_common::util::tokio_util::sync::CancellationToken;
40use risingwave_common::{DATA_DIRECTORY, GIT_SHA, RW_VERSION, STATE_STORE_URL};
41use risingwave_common_heap_profiling::HeapProfiler;
42use risingwave_common_service::{MetricsManager, ObserverManager, TracingExtractLayer};
43use risingwave_connector::source::iceberg::GLOBAL_ICEBERG_SCAN_METRICS;
44use risingwave_connector::source::monitor::GLOBAL_SOURCE_METRICS;
45use risingwave_dml::dml_manager::DmlManager;
46use risingwave_pb::common::WorkerType;
47use risingwave_pb::common::worker_node::Property;
48use risingwave_pb::compute::config_service_server::ConfigServiceServer;
49use risingwave_pb::health::health_server::HealthServer;
50use risingwave_pb::monitor_service::monitor_service_server::MonitorServiceServer;
51use risingwave_pb::stream_service::stream_service_server::StreamServiceServer;
52use risingwave_pb::task_service::batch_exchange_service_server::BatchExchangeServiceServer;
53use risingwave_pb::task_service::stream_exchange_service_server::StreamExchangeServiceServer;
54use risingwave_pb::task_service::task_service_server::TaskServiceServer;
55use risingwave_rpc_client::{ComputeClientPool, MetaClient};
56use risingwave_storage::StateStoreImpl;
57use risingwave_storage::hummock::MemoryLimiter;
58use risingwave_storage::hummock::compactor::{
59    CompactionExecutor, CompactorContext, new_compaction_await_tree_reg_ref, start_compactor,
60};
61use risingwave_storage::hummock::hummock_meta_client::MonitoredHummockMetaClient;
62use risingwave_storage::hummock::utils::HummockMemoryCollector;
63use risingwave_storage::monitor::{
64    GLOBAL_COMPACTOR_METRICS, GLOBAL_HUMMOCK_METRICS, GLOBAL_OBJECT_STORE_METRICS,
65    global_hummock_state_store_metrics, global_storage_metrics, monitor_cache,
66};
67use risingwave_storage::opts::StorageOpts;
68use risingwave_stream::executor::monitor::global_streaming_metrics;
69use risingwave_stream::task::{LocalStreamManager, StreamEnvironment};
70use tokio::sync::oneshot::Sender;
71use tokio::task::JoinHandle;
72use tower::Layer;
73
74use crate::ComputeNodeOpts;
75use crate::memory::config::{
76    MIN_COMPUTE_MEMORY_MB, batch_mem_limit, reserve_memory_bytes, storage_memory_config,
77};
78use crate::memory::manager::{MemoryManager, MemoryManagerConfig};
79use crate::observer::observer_manager::ComputeObserverNode;
80use crate::rpc::service::batch_exchange_service::BatchExchangeServiceImpl;
81use crate::rpc::service::config_service::ConfigServiceImpl;
82use crate::rpc::service::health_service::HealthServiceImpl;
83use crate::rpc::service::monitor_service::{AwaitTreeMiddlewareLayer, MonitorServiceImpl};
84use crate::rpc::service::stream_exchange_service::{
85    GLOBAL_STREAM_EXCHANGE_SERVICE_METRICS, StreamExchangeServiceImpl,
86};
87use crate::rpc::service::stream_service::StreamServiceImpl;
88use crate::telemetry::ComputeTelemetryCreator;
89/// Bootstraps the compute-node.
90///
91/// Returns when the `shutdown` token is triggered.
92pub async fn compute_node_serve(
93    listen_addr: SocketAddr,
94    advertise_addr: HostAddr,
95    opts: Arc<ComputeNodeOpts>,
96    shutdown: CancellationToken,
97) {
98    // Load the configuration.
99    let config = Arc::new(load_config(&opts.config_path, &*opts));
100    info!("Starting compute node",);
101    info!("> config: {:?}", &*config);
102    info!(
103        "> debug assertions: {}",
104        if cfg!(debug_assertions) { "on" } else { "off" }
105    );
106    info!("> version: {} ({})", RW_VERSION, GIT_SHA);
107
108    // Initialize all the configs
109    let stream_config = Arc::new(config.streaming.clone());
110    let batch_config = Arc::new(config.batch.clone());
111
112    // Initialize operator lru cache global sequencer args.
113    init_global_sequencer_args(
114        config
115            .streaming
116            .developer
117            .memory_controller_sequence_tls_step,
118        config
119            .streaming
120            .developer
121            .memory_controller_sequence_tls_lag,
122    );
123
124    // Register to the cluster. We're not ready to serve until activate is called.
125    let (meta_client, system_params) = MetaClient::register_new(
126        opts.meta_address.clone(),
127        WorkerType::ComputeNode,
128        &advertise_addr,
129        Property {
130            parallelism: opts.parallelism as u32,
131            is_streaming: opts.role.for_streaming(),
132            is_serving: opts.role.for_serving(),
133            internal_rpc_host_addr: "".to_owned(),
134            resource_group: Some(opts.resource_group.clone()),
135            is_iceberg_compactor: false,
136        },
137        Arc::new(config.meta.clone()),
138    )
139    .await;
140    // TODO(shutdown): remove this as there's no need to gracefully shutdown the sub-tasks.
141    let mut sub_tasks: Vec<(JoinHandle<()>, Sender<()>)> = vec![];
142    sub_tasks.push(MetaClient::start_heartbeat_loop(
143        meta_client.clone(),
144        Duration::from_millis(config.server.heartbeat_interval_ms as u64),
145    ));
146
147    let state_store_url = system_params.state_store();
148    let data_directory = system_params.data_directory();
149
150    let embedded_compactor_enabled =
151        embedded_compactor_enabled(state_store_url, config.storage.disable_remote_compactor);
152
153    let (reserved_memory_bytes, non_reserved_memory_bytes) = reserve_memory_bytes(&opts);
154    let storage_memory_config = storage_memory_config(
155        non_reserved_memory_bytes,
156        embedded_compactor_enabled,
157        &config.storage,
158        !opts.role.for_streaming(),
159    );
160
161    let storage_memory_bytes = total_storage_memory_limit_bytes(&storage_memory_config);
162    let compute_memory_bytes = validate_compute_node_memory_config(
163        opts.total_memory_bytes,
164        reserved_memory_bytes,
165        storage_memory_bytes,
166    );
167    print_memory_config(
168        opts.total_memory_bytes,
169        compute_memory_bytes,
170        storage_memory_bytes,
171        &storage_memory_config,
172        embedded_compactor_enabled,
173        reserved_memory_bytes,
174    );
175
176    let storage_opts = Arc::new(StorageOpts::from((
177        &*config,
178        &system_params,
179        &storage_memory_config,
180    )));
181
182    let worker_id = meta_client.worker_id();
183    info!("Assigned worker node id {}", worker_id);
184
185    // Initialize the metrics subsystem.
186    let source_metrics = Arc::new(GLOBAL_SOURCE_METRICS.clone());
187    let hummock_metrics = Arc::new(GLOBAL_HUMMOCK_METRICS.clone());
188    let streaming_metrics = Arc::new(global_streaming_metrics(config.server.metrics_level));
189    let batch_executor_metrics = Arc::new(GLOBAL_BATCH_EXECUTOR_METRICS.clone());
190    let batch_manager_metrics = Arc::new(GLOBAL_BATCH_MANAGER_METRICS.clone());
191    let stream_exchange_srv_metrics = Arc::new(GLOBAL_STREAM_EXCHANGE_SERVICE_METRICS.clone());
192    let batch_spill_metrics = Arc::new(GLOBAL_BATCH_SPILL_METRICS.clone());
193    let iceberg_scan_metrics = Arc::new(GLOBAL_ICEBERG_SCAN_METRICS.clone());
194
195    // Initialize state store.
196    let state_store_metrics = Arc::new(global_hummock_state_store_metrics(
197        config.server.metrics_level,
198    ));
199    let object_store_metrics = Arc::new(GLOBAL_OBJECT_STORE_METRICS.clone());
200    let storage_metrics = Arc::new(global_storage_metrics(config.server.metrics_level));
201    let compactor_metrics = Arc::new(GLOBAL_COMPACTOR_METRICS.clone());
202    let hummock_meta_client = Arc::new(MonitoredHummockMetaClient::new(
203        meta_client.clone(),
204        hummock_metrics.clone(),
205    ));
206
207    let await_tree_config = match &config.streaming.async_stack_trace {
208        AsyncStackTraceOption::Off => None,
209        c => await_tree::ConfigBuilder::default()
210            .verbose(c.is_verbose().unwrap())
211            .build()
212            .ok(),
213    };
214    // Store the state_store_url in a static OnceLock for later use in JNI crate
215    // Check the return value and if the variable is set, assert that the value is the same.
216    if let Err(existing_url) = STATE_STORE_URL.set(state_store_url.to_owned()) {
217        assert_eq!(
218            existing_url, state_store_url,
219            "STATE_STORE_URL already set with different value"
220        );
221    }
222
223    // Store the data_directory in a static OnceLock for later use in JNI crate
224    // To be extra safe, check the return value and if the variable is set, assert that the value is the same
225    if let Err(existing_dir) = DATA_DIRECTORY.set(data_directory.to_owned()) {
226        assert_eq!(
227            existing_dir, data_directory,
228            "DATA_DIRECTORY already set with different value"
229        );
230    }
231    LicenseManager::get().refresh(system_params.license_key());
232    let state_store = Box::pin(StateStoreImpl::new(
233        state_store_url,
234        storage_opts.clone(),
235        hummock_meta_client.clone(),
236        state_store_metrics.clone(),
237        object_store_metrics,
238        storage_metrics.clone(),
239        compactor_metrics.clone(),
240        await_tree_config.clone(),
241        system_params.use_new_object_prefix_strategy(),
242    ))
243    .await
244    .unwrap();
245
246    LocalSecretManager::init(
247        opts.temp_secret_file_dir.clone(),
248        meta_client.cluster_id().to_owned(),
249        worker_id,
250    );
251
252    // Initialize observer manager.
253    let batch_client_pool = Arc::new(ComputeClientPool::new(
254        config.batch_exchange_connection_pool_size(),
255        config.batch.developer.compute_client_config.clone(),
256    ));
257    let system_params_manager = Arc::new(LocalSystemParamsManager::new(system_params.clone()));
258    let compute_observer_node =
259        ComputeObserverNode::new(system_params_manager.clone(), batch_client_pool.clone());
260    let observer_manager =
261        ObserverManager::new_with_meta_client(meta_client.clone(), compute_observer_node).await;
262    observer_manager.start().await;
263
264    if let Some(storage) = state_store.as_hummock() {
265        if embedded_compactor_enabled {
266            tracing::info!("start embedded compactor");
267            let memory_limiter = Arc::new(MemoryLimiter::new(
268                storage_opts.compactor_memory_limit_mb as u64 * 1024 * 1024 / 2,
269            ));
270
271            let compaction_executor = Arc::new(CompactionExecutor::new(Some(1)));
272            let compactor_context = CompactorContext {
273                storage_opts,
274                sstable_store: storage.sstable_store(),
275                compactor_metrics: compactor_metrics.clone(),
276                is_share_buffer_compact: false,
277                compaction_executor,
278                memory_limiter,
279
280                task_progress_manager: Default::default(),
281                await_tree_reg: await_tree_config
282                    .clone()
283                    .map(new_compaction_await_tree_reg_ref),
284            };
285
286            let (handle, shutdown_sender) = start_compactor(
287                compactor_context,
288                hummock_meta_client.clone(),
289                storage.object_id_manager().clone(),
290                storage.compaction_catalog_manager_ref(),
291            );
292            sub_tasks.push((handle, shutdown_sender));
293        }
294        let flush_limiter = storage.get_memory_limiter();
295        let memory_collector = Arc::new(HummockMemoryCollector::new(
296            storage.sstable_store(),
297            flush_limiter,
298            storage_memory_config,
299        ));
300        monitor_cache(memory_collector);
301        let backup_reader = storage.backup_reader();
302        let system_params_mgr = system_params_manager.clone();
303        tokio::spawn(async move {
304            backup_reader
305                .watch_config_change(system_params_mgr.watch_params())
306                .await;
307        });
308    }
309
310    // Initialize the managers.
311    let batch_mgr = Arc::new(BatchManager::new(
312        config.batch.clone(),
313        batch_manager_metrics,
314        batch_mem_limit(compute_memory_bytes, opts.role.for_serving()),
315    ));
316
317    let target_memory = if let Some(v) = opts.memory_manager_target_bytes {
318        v
319    } else {
320        compute_memory_bytes + storage_memory_bytes
321    };
322
323    let memory_mgr = MemoryManager::new(MemoryManagerConfig {
324        target_memory,
325        threshold_aggressive: config
326            .streaming
327            .developer
328            .memory_controller_threshold_aggressive,
329        threshold_graceful: config
330            .streaming
331            .developer
332            .memory_controller_threshold_graceful,
333        threshold_stable: config
334            .streaming
335            .developer
336            .memory_controller_threshold_stable,
337        eviction_factor_stable: config
338            .streaming
339            .developer
340            .memory_controller_eviction_factor_stable,
341        eviction_factor_graceful: config
342            .streaming
343            .developer
344            .memory_controller_eviction_factor_graceful,
345        eviction_factor_aggressive: config
346            .streaming
347            .developer
348            .memory_controller_eviction_factor_aggressive,
349        metrics: streaming_metrics.clone(),
350    });
351
352    // Run a background memory manager
353    tokio::spawn(
354        memory_mgr.clone().run(Duration::from_millis(
355            config
356                .streaming
357                .developer
358                .memory_controller_update_interval_ms as _,
359        )),
360    );
361
362    let heap_profiler = HeapProfiler::new(
363        opts.total_memory_bytes,
364        config.server.heap_profiling.clone(),
365    );
366    // Run a background heap profiler
367    heap_profiler.start();
368
369    let dml_mgr = Arc::new(DmlManager::new(
370        worker_id,
371        config.streaming.developer.dml_channel_initial_permits,
372    ));
373
374    // Initialize batch environment.
375    let batch_env = BatchEnvironment::new(
376        batch_mgr.clone(),
377        advertise_addr.clone(),
378        batch_config,
379        worker_id,
380        state_store.clone(),
381        batch_executor_metrics.clone(),
382        batch_client_pool,
383        dml_mgr.clone(),
384        source_metrics.clone(),
385        batch_spill_metrics.clone(),
386        iceberg_scan_metrics.clone(),
387        config.server.metrics_level,
388    );
389
390    // Initialize the streaming environment.
391    let stream_client_pool = Arc::new(ComputeClientPool::new(
392        config.streaming_exchange_connection_pool_size(),
393        config.streaming.developer.compute_client_config.clone(),
394    ));
395    let stream_env = StreamEnvironment::new(
396        advertise_addr.clone(),
397        stream_config,
398        worker_id,
399        state_store.clone(),
400        dml_mgr,
401        system_params_manager.clone(),
402        source_metrics,
403        meta_client.clone(),
404        stream_client_pool,
405    );
406
407    let stream_mgr = LocalStreamManager::new(
408        stream_env.clone(),
409        streaming_metrics.clone(),
410        await_tree_config.clone(),
411        memory_mgr.get_watermark_sequence(),
412    );
413
414    // Boot the runtime gRPC services.
415    let batch_srv = BatchServiceImpl::new(batch_mgr.clone(), batch_env);
416    let batch_exchange_srv = BatchExchangeServiceImpl::new(batch_mgr.clone());
417    let stream_exchange_srv =
418        StreamExchangeServiceImpl::new(stream_mgr.clone(), stream_exchange_srv_metrics);
419    let stream_srv = StreamServiceImpl::new(stream_mgr.clone(), stream_env.clone());
420    let (meta_cache, block_cache) = if let Some(hummock) = state_store.as_hummock() {
421        (
422            Some(hummock.sstable_store().meta_cache().clone()),
423            Some(hummock.sstable_store().block_cache().clone()),
424        )
425    } else {
426        (None, None)
427    };
428    let monitor_srv = MonitorServiceImpl::new(
429        stream_mgr.clone(),
430        config.server.clone(),
431        meta_cache.clone(),
432        block_cache.clone(),
433    );
434    let config_srv = ConfigServiceImpl::new(batch_mgr, stream_mgr.clone(), meta_cache, block_cache);
435    let health_srv = HealthServiceImpl::new();
436
437    let telemetry_manager = TelemetryManager::new(
438        Arc::new(meta_client.clone()),
439        Arc::new(ComputeTelemetryCreator::new()),
440    );
441
442    // if the toml config file or env variable disables telemetry, do not watch system params change
443    // because if any of configs disable telemetry, we should never start it
444    if config.server.telemetry_enabled && telemetry_env_enabled() {
445        sub_tasks.push(telemetry_manager.start().await);
446    } else {
447        tracing::info!("Telemetry didn't start due to config");
448    }
449
450    // Clean up the spill directory.
451    #[cfg(not(madsim))]
452    if config.batch.enable_spill {
453        SpillOp::clean_spill_directory().await.unwrap();
454    }
455
456    let server = tonic::transport::Server::builder()
457        .initial_connection_window_size(MAX_CONNECTION_WINDOW_SIZE)
458        .initial_stream_window_size(STREAM_WINDOW_SIZE)
459        .http2_max_pending_accept_reset_streams(Some(config.server.grpc_max_reset_stream as usize))
460        .layer(TracingExtractLayer::new())
461        // XXX: unlimit the max message size to allow arbitrary large SQL input.
462        .add_service(TaskServiceServer::new(batch_srv).max_decoding_message_size(usize::MAX))
463        .add_service(
464            BatchExchangeServiceServer::new(batch_exchange_srv)
465                .max_decoding_message_size(usize::MAX),
466        )
467        .add_service(
468            StreamExchangeServiceServer::new(stream_exchange_srv)
469                .max_decoding_message_size(usize::MAX),
470        )
471        .add_service({
472            let await_tree_reg = stream_srv.mgr.await_tree_reg().cloned();
473            let srv = StreamServiceServer::new(stream_srv).max_decoding_message_size(usize::MAX);
474            #[cfg(madsim)]
475            {
476                srv
477            }
478            #[cfg(not(madsim))]
479            {
480                AwaitTreeMiddlewareLayer::new_optional(await_tree_reg).layer(srv)
481            }
482        })
483        .add_service(MonitorServiceServer::new(monitor_srv))
484        .add_service(ConfigServiceServer::new(config_srv))
485        .add_service(HealthServer::new(health_srv))
486        .monitored_serve_with_shutdown(
487            listen_addr,
488            "grpc-compute-node-service",
489            TcpConfig {
490                tcp_nodelay: true,
491                keepalive_duration: None,
492            },
493            shutdown.clone().cancelled_owned(),
494        );
495    let _server_handle = tokio::spawn(server);
496
497    // Boot metrics service.
498    if config.server.metrics_level > MetricLevel::Disabled {
499        MetricsManager::boot_metrics_service(opts.prometheus_listener_addr.clone());
500    }
501
502    // All set, let the meta service know we're ready.
503    meta_client.activate(&advertise_addr).await.unwrap();
504    // Wait for the shutdown signal.
505    shutdown.cancelled().await;
506
507    // Unregister from the meta service, then...
508    // - batch queries will not be scheduled to this compute node,
509    // - streaming actors will not be scheduled to this compute node after next recovery.
510    meta_client.try_unregister().await;
511    // Shutdown the streaming manager.
512    let _ = stream_mgr.shutdown().await;
513
514    // NOTE(shutdown): We can't simply join the tonic server here because it only returns when all
515    // existing connections are closed, while we have long-running streaming calls that never
516    // close. From the other side, there's also no need to gracefully shutdown them if we have
517    // unregistered from the meta service.
518}
519
520/// Check whether the compute node has enough memory to perform computing tasks. Apart from storage,
521/// it is recommended to reserve at least `MIN_COMPUTE_MEMORY_MB` for computing and
522/// `SYSTEM_RESERVED_MEMORY_PROPORTION` of total memory for other system usage. If the requirement
523/// is not met, we will print out a warning and enforce the memory used for computing tasks as
524/// `MIN_COMPUTE_MEMORY_MB`.
525fn validate_compute_node_memory_config(
526    cn_total_memory_bytes: usize,
527    reserved_memory_bytes: usize,
528    storage_memory_bytes: usize,
529) -> usize {
530    if storage_memory_bytes > cn_total_memory_bytes {
531        tracing::warn!(
532            "The storage memory exceeds the total compute node memory:\nTotal compute node memory: {}\nStorage memory: {}\nWe recommend that at least 4 GiB memory should be reserved for RisingWave. Please increase the total compute node memory or decrease the storage memory in configurations.",
533            convert(cn_total_memory_bytes as _),
534            convert(storage_memory_bytes as _)
535        );
536        MIN_COMPUTE_MEMORY_MB << 20
537    } else if storage_memory_bytes + (MIN_COMPUTE_MEMORY_MB << 20) + reserved_memory_bytes
538        >= cn_total_memory_bytes
539    {
540        tracing::warn!(
541            "No enough memory for computing and other system usage:\nTotal compute node memory: {}\nStorage memory: {}\nWe recommend that at least 4 GiB memory should be reserved for RisingWave. Please increase the total compute node memory or decrease the storage memory in configurations.",
542            convert(cn_total_memory_bytes as _),
543            convert(storage_memory_bytes as _)
544        );
545        MIN_COMPUTE_MEMORY_MB << 20
546    } else {
547        cn_total_memory_bytes - storage_memory_bytes - reserved_memory_bytes
548    }
549}
550
551/// The maximal memory that storage components may use based on the configurations in bytes. Note
552/// that this is the total storage memory for one compute node instead of the whole cluster.
553fn total_storage_memory_limit_bytes(storage_memory_config: &StorageMemoryConfig) -> usize {
554    let total_storage_memory_mb = storage_memory_config.block_cache_capacity_mb
555        + storage_memory_config.meta_cache_capacity_mb
556        + storage_memory_config.shared_buffer_capacity_mb
557        + storage_memory_config.compactor_memory_limit_mb;
558    total_storage_memory_mb << 20
559}
560
561/// Checks whether an embedded compactor starts with a compute node.
562fn embedded_compactor_enabled(state_store_url: &str, disable_remote_compactor: bool) -> bool {
563    // Always start an embedded compactor if the state store is in-memory.
564    state_store_url.starts_with("hummock+memory")
565        || state_store_url.starts_with("hummock+disk")
566        || disable_remote_compactor
567}
568
569// Print out the memory outline of the compute node.
570fn print_memory_config(
571    cn_total_memory_bytes: usize,
572    compute_memory_bytes: usize,
573    storage_memory_bytes: usize,
574    storage_memory_config: &StorageMemoryConfig,
575    embedded_compactor_enabled: bool,
576    reserved_memory_bytes: usize,
577) {
578    let memory_config = format!(
579        "Memory outline:\n\
580        > total_memory: {}\n\
581        >     storage_memory: {}\n\
582        >         block_cache_capacity: {}\n\
583        >         meta_cache_capacity: {}\n\
584        >         shared_buffer_capacity: {}\n\
585        >         compactor_memory_limit: {}\n\
586        >     compute_memory: {}\n\
587        >     reserved_memory: {}",
588        convert(cn_total_memory_bytes as _),
589        convert(storage_memory_bytes as _),
590        convert((storage_memory_config.block_cache_capacity_mb << 20) as _),
591        convert((storage_memory_config.meta_cache_capacity_mb << 20) as _),
592        convert((storage_memory_config.shared_buffer_capacity_mb << 20) as _),
593        if embedded_compactor_enabled {
594            convert((storage_memory_config.compactor_memory_limit_mb << 20) as _)
595        } else {
596            "Not enabled".to_owned()
597        },
598        convert(compute_memory_bytes as _),
599        convert(reserved_memory_bytes as _),
600    );
601    info!("{}", memory_config);
602}