Skip to content

Commit

Permalink
refactor(metrics): unify storage_metric_level & streaming_metric_leve…
Browse files Browse the repository at this point in the history
…l to 'metrics_level' (#12099)
  • Loading branch information
fuyufjh authored Sep 7, 2023
1 parent 5f75f9b commit b2dc4eb
Show file tree
Hide file tree
Showing 32 changed files with 102 additions and 124 deletions.
4 changes: 2 additions & 2 deletions ci/scripts/standalone-utils.sh
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ start_standalone() {
--listen-addr 127.0.0.1:5688 \
--prometheus-listener-addr 127.0.0.1:1222 \
--advertise-addr 127.0.0.1:5688 \
--metrics-level 1 \
--metrics-level info \
--async-stack-trace verbose \
--connector-rpc-endpoint 127.0.0.1:50051 \
--parallelism 4 \
Expand All @@ -36,7 +36,7 @@ start_standalone() {
--advertise-addr 127.0.0.1:4566 \
--prometheus-listener-addr 127.0.0.1:2222 \
--health-check-listener-addr 127.0.0.1:6786 \
--metrics-level 1 \
--metrics-level info \
--meta-addr http://127.0.0.1:5690" >"$1" 2>&1
}

Expand Down
2 changes: 1 addition & 1 deletion docker/dashboards/risingwave-dev-dashboard.json

Large diffs are not rendered by default.

6 changes: 3 additions & 3 deletions docker/docker-compose-with-hdfs.yml
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ services:
- "--prometheus-listener-addr"
- "0.0.0.0:1260"
- "--metrics-level"
- "1"
- "info"
- "--meta-address"
- "http://meta-node-0:5690"
- "--config-path"
Expand Down Expand Up @@ -47,7 +47,7 @@ services:
- "--prometheus-listener-addr"
- "0.0.0.0:1222"
- "--metrics-level"
- "1"
- "info"
- "--meta-address"
- "http://meta-node-0:5690"
- "--connector-rpc-endpoint"
Expand Down Expand Up @@ -137,7 +137,7 @@ services:
- "--prometheus-listener-addr"
- "0.0.0.0:2222"
- "--metrics-level"
- "1"
- "info"
expose:
- "4566"
ports:
Expand Down
6 changes: 3 additions & 3 deletions docker/docker-compose.yml
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ services:
- "--prometheus-listener-addr"
- "0.0.0.0:1260"
- "--metrics-level"
- "1"
- "info"
- "--meta-address"
- "http://meta-node-0:5690"
- "--config-path"
Expand Down Expand Up @@ -49,7 +49,7 @@ services:
- "--prometheus-listener-addr"
- "0.0.0.0:1222"
- "--metrics-level"
- "1"
- "info"
- "--meta-address"
- "http://meta-node-0:5690"
- "--connector-rpc-endpoint"
Expand Down Expand Up @@ -140,7 +140,7 @@ services:
- "--prometheus-listener-addr"
- "0.0.0.0:2222"
- "--metrics-level"
- "1"
- "info"
expose:
- "4566"
ports:
Expand Down
2 changes: 1 addition & 1 deletion grafana/risingwave-dev-dashboard.dashboard.py
Original file line number Diff line number Diff line change
Expand Up @@ -1911,7 +1911,7 @@ def section_hummock(panels):
],
),
panels.timeseries_percentage(
"Bloom-Filter Miss Rate",
"Bloom-Filter Positive Rate",
"Positive / Total",
[
panels.target(
Expand Down
2 changes: 1 addition & 1 deletion grafana/risingwave-dev-dashboard.json

Large diffs are not rendered by default.

5 changes: 2 additions & 3 deletions risedev.yml
Original file line number Diff line number Diff line change
Expand Up @@ -815,7 +815,7 @@ profile:
- use: compactor

ci-backfill:
config-path: "src/config/backfill.toml"
config-path: "src/config/ci-backfill.toml"
steps:
- use: minio
- use: etcd
Expand Down Expand Up @@ -1167,7 +1167,6 @@ template:

bucket: risingwave-test


# aws-s3 is a placeholder service to provide configurations
aws-s3:
# Id to be picked-up by services
Expand Down Expand Up @@ -1257,4 +1256,4 @@ template:
port: 6379

# address of redis
address: "127.0.0.1"
address: "127.0.0.1"
4 changes: 2 additions & 2 deletions src/cmd_all/scripts/standalone-demo-dev.sh
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ cargo run -p risingwave_cmd_all \
--listen-addr 127.0.0.1:5688 \
--prometheus-listener-addr 127.0.0.1:1222 \
--advertise-addr 127.0.0.1:5688 \
--metrics-level 1 \
--metrics-level info \
--async-stack-trace verbose \
--connector-rpc-endpoint 127.0.0.1:50051 \
--parallelism 4 \
Expand All @@ -35,5 +35,5 @@ cargo run -p risingwave_cmd_all \
--advertise-addr 127.0.0.1:4566 \
--prometheus-listener-addr 127.0.0.1:2222 \
--health-check-listener-addr 127.0.0.1:6786 \
--metrics-level 1 \
--metrics-level info \
--meta-addr http://127.0.0.1:5690"
4 changes: 2 additions & 2 deletions src/cmd_all/scripts/standalone-demo-full.sh
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ start_standalone() {
--listen-addr 127.0.0.1:5688 \
--prometheus-listener-addr 127.0.0.1:1222 \
--advertise-addr 127.0.0.1:5688 \
--metrics-level 1 \
--metrics-level info \
--async-stack-trace verbose \
--connector-rpc-endpoint 127.0.0.1:50051 \
--parallelism 4 \
Expand All @@ -42,7 +42,7 @@ start_standalone() {
--advertise-addr 127.0.0.1:4566 \
--prometheus-listener-addr 127.0.0.1:2222 \
--health-check-listener-addr 127.0.0.1:6786 \
--metrics-level 1 \
--metrics-level info \
--meta-addr http://127.0.0.1:5690"
}

Expand Down
31 changes: 6 additions & 25 deletions src/common/src/config.rs
Original file line number Diff line number Diff line change
Expand Up @@ -359,11 +359,9 @@ pub struct ServerConfig {
#[serde(default = "default::server::connection_pool_size")]
pub connection_pool_size: u16,

#[serde(default = "default::server::metrics_level")]
/// Used for control the metrics level, similar to log level.
/// 0 = close metrics
/// >0 = open metrics
pub metrics_level: u32,
#[serde(default = "default::server::metrics_level")]
pub metrics_level: MetricLevel,

#[serde(default = "default::server::telemetry_enabled")]
pub telemetry_enabled: bool,
Expand Down Expand Up @@ -420,9 +418,6 @@ pub struct StreamingConfig {
#[serde(default = "default::streaming::unique_user_stream_errors")]
pub unique_user_stream_errors: usize,

#[serde(default = "default::streaming::streaming_metric_level")]
pub streaming_metric_level: MetricLevel,

#[serde(default, flatten)]
pub unrecognized: Unrecognized<Self>,
}
Expand Down Expand Up @@ -568,10 +563,6 @@ pub struct StorageConfig {
pub compact_iter_recreate_timeout_ms: u64,
#[serde(default = "default::storage::compactor_max_sst_size")]
pub compactor_max_sst_size: u64,

#[serde(default = "default::storage::storage_metric_level")]
pub storage_metric_level: MetricLevel,

#[serde(default, flatten)]
pub unrecognized: Unrecognized<Self>,
}
Expand Down Expand Up @@ -917,7 +908,7 @@ pub mod default {
}

pub mod server {
use crate::config::AutoDumpHeapProfileConfig;
use crate::config::{AutoDumpHeapProfileConfig, MetricLevel};

pub fn heartbeat_interval_ms() -> u32 {
1000
Expand All @@ -927,8 +918,8 @@ pub mod default {
16
}

pub fn metrics_level() -> u32 {
0
pub fn metrics_level() -> MetricLevel {
MetricLevel::Info
}

pub fn telemetry_enabled() -> bool {
Expand All @@ -941,8 +932,6 @@ pub mod default {
}

pub mod storage {
use crate::config::MetricLevel;

pub fn share_buffers_sync_parallelism() -> u32 {
1
}
Expand Down Expand Up @@ -1047,14 +1036,10 @@ pub mod default {
pub fn compactor_max_sst_size() -> u64 {
512 * 1024 * 1024 // 512m
}

pub fn storage_metric_level() -> MetricLevel {
MetricLevel::Info
}
}

pub mod streaming {
use crate::config::{AsyncStackTraceOption, MetricLevel};
use crate::config::AsyncStackTraceOption;

pub fn in_flight_barrier_nums() -> usize {
// quick fix
Expand All @@ -1069,10 +1054,6 @@ pub mod default {
pub fn unique_user_stream_errors() -> usize {
10
}

pub fn streaming_metric_level() -> MetricLevel {
MetricLevel::Info
}
}

pub mod file_cache {
Expand Down
4 changes: 2 additions & 2 deletions src/common/src/metrics/relabeled_metric.rs
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ impl RelabeledHistogramVec {
for label in relabeled_vals.iter_mut().take(self.relabel_num) {
*label = "";
}
self.metric.with_label_values(&relabeled_vals);
return self.metric.with_label_values(&relabeled_vals);
}
self.metric.with_label_values(vals)
}
Expand Down Expand Up @@ -125,7 +125,7 @@ impl RelabeledCounterVec {
for label in relabeled_vals.iter_mut().take(self.relabel_num) {
*label = "";
}
self.metric.with_label_values(&relabeled_vals);
return self.metric.with_label_values(&relabeled_vals);
}
self.metric.with_label_values(vals)
}
Expand Down
4 changes: 2 additions & 2 deletions src/compute/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ use std::future::Future;
use std::pin::Pin;

use clap::{Parser, ValueEnum};
use risingwave_common::config::{AsyncStackTraceOption, OverrideConfig};
use risingwave_common::config::{AsyncStackTraceOption, MetricLevel, OverrideConfig};
use risingwave_common::util::resource_util::cpu::total_cpu_available;
use risingwave_common::util::resource_util::memory::total_memory_available_bytes;
use serde::{Deserialize, Serialize};
Expand Down Expand Up @@ -103,7 +103,7 @@ pub struct ComputeNodeOpts {
/// >0 = enable metrics
#[clap(long, env = "RW_METRICS_LEVEL")]
#[override_opts(path = server.metrics_level)]
pub metrics_level: Option<u32>,
pub metrics_level: Option<MetricLevel>,

/// Path to data file cache data directory.
/// Left empty to disable file cache.
Expand Down
14 changes: 6 additions & 8 deletions src/compute/src/server.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@ use risingwave_batch::monitor::{
use risingwave_batch::rpc::service::task_service::BatchServiceImpl;
use risingwave_batch::task::{BatchEnvironment, BatchManager};
use risingwave_common::config::{
load_config, AsyncStackTraceOption, StorageMemoryConfig, MAX_CONNECTION_WINDOW_SIZE,
STREAM_WINDOW_SIZE,
load_config, AsyncStackTraceOption, MetricLevel, StorageMemoryConfig,
MAX_CONNECTION_WINDOW_SIZE, STREAM_WINDOW_SIZE,
};
use risingwave_common::monitor::connection::{RouterExt, TcpConfig};
use risingwave_common::system_param::local_manager::LocalSystemParamsManager;
Expand Down Expand Up @@ -169,20 +169,18 @@ pub async fn compute_node_serve(
// Initialize the metrics subsystem.
let source_metrics = Arc::new(GLOBAL_SOURCE_METRICS.clone());
let hummock_metrics = Arc::new(GLOBAL_HUMMOCK_METRICS.clone());
let streaming_metrics = Arc::new(global_streaming_metrics(
config.streaming.streaming_metric_level,
));
let streaming_metrics = Arc::new(global_streaming_metrics(config.server.metrics_level));
let batch_task_metrics = Arc::new(GLOBAL_BATCH_TASK_METRICS.clone());
let batch_executor_metrics = Arc::new(GLOBAL_BATCH_EXECUTOR_METRICS.clone());
let batch_manager_metrics = GLOBAL_BATCH_MANAGER_METRICS.clone();
let exchange_srv_metrics = Arc::new(GLOBAL_EXCHANGE_SERVICE_METRICS.clone());

// Initialize state store.
let state_store_metrics = Arc::new(global_hummock_state_store_metrics(
config.storage.storage_metric_level,
config.server.metrics_level,
));
let object_store_metrics = Arc::new(GLOBAL_OBJECT_STORE_METRICS.clone());
let storage_metrics = Arc::new(global_storage_metrics(config.storage.storage_metric_level));
let storage_metrics = Arc::new(global_storage_metrics(config.server.metrics_level));
let compactor_metrics = Arc::new(GLOBAL_COMPACTOR_METRICS.clone());
let hummock_meta_client = Arc::new(MonitoredHummockMetaClient::new(
meta_client.clone(),
Expand Down Expand Up @@ -454,7 +452,7 @@ pub async fn compute_node_serve(
join_handle_vec.push(join_handle);

// Boot metrics service.
if config.server.metrics_level > 0 {
if config.server.metrics_level > MetricLevel::Disabled {
MetricsManager::boot_metrics_service(opts.prometheus_listener_addr.clone());
}

Expand Down
File renamed without changes.
3 changes: 2 additions & 1 deletion src/config/ci-sim.toml
Original file line number Diff line number Diff line change
@@ -1,9 +1,10 @@
[server]
telemetry_enabled = false
metrics_level = "Disabled"

[system]
telemetry_enabled = false
max_concurrent_creating_streaming_jobs = 0

[meta]
meta_leader_lease_secs = 10
meta_leader_lease_secs = 10
4 changes: 1 addition & 3 deletions src/config/example.toml
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
[server]
heartbeat_interval_ms = 1000
connection_pool_size = 16
metrics_level = 0
metrics_level = "Info"
telemetry_enabled = true

[server.auto_dump_heap_profile]
Expand Down Expand Up @@ -67,7 +67,6 @@ batch_chunk_size = 1024
in_flight_barrier_nums = 10000
async_stack_trace = "ReleaseVerbose"
unique_user_stream_errors = 10
streaming_metric_level = "Info"

[streaming.developer]
stream_enable_executor_row_count = false
Expand Down Expand Up @@ -107,7 +106,6 @@ object_store_req_retry_max_attempts = 8
compactor_max_sst_key_count = 2097152
compact_iter_recreate_timeout_ms = 600000
compactor_max_sst_size = 536870912
storage_metric_level = "Info"

[storage.data_file_cache]
dir = ""
Expand Down
2 changes: 1 addition & 1 deletion src/config/full-iceberg-bench.toml
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
[server]
heartbeat_interval_ms = 1000
connection_pool_size = 16
metrics_level = 0
metrics_level = "Info"
telemetry_enabled = true

[server.auto_dump_heap_profile]
Expand Down
4 changes: 2 additions & 2 deletions src/frontend/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ pub use planner::Planner;
mod scheduler;
pub mod session;
mod stream_fragmenter;
use risingwave_common::config::OverrideConfig;
use risingwave_common::config::{MetricLevel, OverrideConfig};
pub use stream_fragmenter::build_graph;
mod utils;
pub use utils::{explain_stream_graph, WithOptions};
Expand Down Expand Up @@ -129,7 +129,7 @@ pub struct FrontendOpts {
/// >0 = enable metrics
#[clap(long, env = "RW_METRICS_LEVEL")]
#[override_opts(path = server.metrics_level)]
pub metrics_level: Option<u32>,
pub metrics_level: Option<MetricLevel>,

#[clap(long, env = "RW_ENABLE_BARRIER_READ")]
#[override_opts(path = batch.enable_barrier_read)]
Expand Down
4 changes: 2 additions & 2 deletions src/frontend/src/session.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ use risingwave_common::catalog::DEFAULT_SCHEMA_NAME;
use risingwave_common::catalog::{
DEFAULT_DATABASE_NAME, DEFAULT_SUPER_USER, DEFAULT_SUPER_USER_ID,
};
use risingwave_common::config::{load_config, BatchConfig, MetaConfig};
use risingwave_common::config::{load_config, BatchConfig, MetaConfig, MetricLevel};
use risingwave_common::error::{ErrorCode, Result, RwError};
use risingwave_common::session_config::{ConfigMap, ConfigReporter, VisibilityMode};
use risingwave_common::system_param::local_manager::LocalSystemParamsManager;
Expand Down Expand Up @@ -277,7 +277,7 @@ impl FrontendEnv {
let frontend_metrics = Arc::new(GLOBAL_FRONTEND_METRICS.clone());
let source_metrics = Arc::new(GLOBAL_SOURCE_METRICS.clone());

if config.server.metrics_level > 0 {
if config.server.metrics_level > MetricLevel::Disabled {
MetricsManager::boot_metrics_service(opts.prometheus_listener_addr.clone());
}

Expand Down
Loading

0 comments on commit b2dc4eb

Please sign in to comment.